You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2016/10/11 20:13:15 UTC

[1/3] hadoop git commit: YARN-5610. Initial code for native services REST API. Contributed by Gour Saha

Repository: hadoop
Updated Branches:
  refs/heads/yarn-native-services 625b92ced -> 4d5ded9f8


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Resource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Resource.java
new file mode 100644
index 0000000..a3780cc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Resource.java
@@ -0,0 +1,149 @@
+/*
+ * 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.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Resource determines the amount of resources (vcores, memory, network, etc.)
+ * usable by a container. This field determines the resource to be applied for
+ * all the containers of a component or application. The resource specified at
+ * the app (or global) level can be overriden at the component level. Only one
+ * of profile OR cpu & memory are exepected. It raises a validation
+ * exception otherwise.
+ **/
+
+@ApiModel(description = "Resource determines the amount of resources (vcores, memory, network, etc.) usable by a container. This field determines the resource to be applied for all the containers of a component or application. The resource specified at the app (or global) level can be overriden at the component level. Only one of profile OR cpu & memory are exepected. It raises a validation exception otherwise.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+public class Resource extends BaseResource {
+  private static final long serialVersionUID = -6431667797380250037L;
+
+  private String profile = null;
+  private Integer cpus = null;
+  private String memory = null;
+
+  /**
+   * Each resource profile has a unique id which is associated with a
+   * cluster-level predefined memory, cpus, etc.
+   **/
+  public Resource profile(String profile) {
+    this.profile = profile;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Each resource profile has a unique id which is associated with a cluster-level predefined memory, cpus, etc.")
+  @JsonProperty("profile")
+  public String getProfile() {
+    return profile;
+  }
+
+  public void setProfile(String profile) {
+    this.profile = profile;
+  }
+
+  /**
+   * Amount of vcores allocated to each container (optional but overrides cpus
+   * in profile if specified).
+   **/
+  public Resource cpus(Integer cpus) {
+    this.cpus = cpus;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Amount of vcores allocated to each container (optional but overrides cpus in profile if specified).")
+  @JsonProperty("cpus")
+  public Integer getCpus() {
+    return cpus;
+  }
+
+  public void setCpus(Integer cpus) {
+    this.cpus = cpus;
+  }
+
+  /**
+   * Amount of memory allocated to each container (optional but overrides memory
+   * in profile if specified). Currently accepts only an integer value and
+   * default unit is in MB.
+   **/
+  public Resource memory(String memory) {
+    this.memory = memory;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Amount of memory allocated to each container (optional but overrides memory in profile if specified). Currently accepts only an integer value and default unit is in MB.")
+  @JsonProperty("memory")
+  public String getMemory() {
+    return memory;
+  }
+
+  public void setMemory(String memory) {
+    this.memory = memory;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Resource resource = (Resource) o;
+    return Objects.equals(this.profile, resource.profile)
+        && Objects.equals(this.cpus, resource.cpus)
+        && Objects.equals(this.memory, resource.memory);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(profile, cpus, memory);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Resource {\n");
+
+    sb.append("    profile: ").append(toIndentedString(profile)).append("\n");
+    sb.append("    cpus: ").append(toIndentedString(cpus)).append("\n");
+    sb.append("    memory: ").append(toIndentedString(memory)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+
+  @Override
+  public Object clone() throws CloneNotSupportedException {
+    return super.clone();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiConstants.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiConstants.java
new file mode 100644
index 0000000..4c16546
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiConstants.java
@@ -0,0 +1,66 @@
+/*
+ * 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.services.utils;
+
+public interface RestApiConstants {
+  String CONTEXT_ROOT = "/services/v1";
+  String APPLICATIONS_API_RESOURCE_PATH = "/applications";
+  String CONTAINERS_API_RESOURCE_PATH = "/containers";
+  String SLIDER_APPMASTER_COMPONENT_NAME = "slider-appmaster";
+  String SLIDER_CONFIG_SCHEMA = "http://example.org/specification/v2.0.0";
+  String METAINFO_SCHEMA_VERSION = "2.1";
+  String COMPONENT_TYPE_YARN_DOCKER = "yarn_docker";
+
+  String DEFAULT_START_CMD = "/bootstrap/privileged-centos6-sshd";
+  String DEFAULT_COMPONENT_NAME = "DEFAULT";
+  String DEFAULT_IMAGE = "centos:centos6";
+  String DEFAULT_NETWORK = "bridge";
+  String DEFAULT_COMMAND_PATH = "/usr/bin/docker";
+  String DEFAULT_USE_NETWORK_SCRIPT = "yes";
+
+  String PLACEHOLDER_APP_NAME = "${APP_NAME}";
+  String PLACEHOLDER_APP_COMPONENT_NAME = "${APP_COMPONENT_NAME}";
+  String PLACEHOLDER_COMPONENT_ID = "${COMPONENT_ID}";
+
+  String PROPERTY_REST_SERVICE_HOST = "REST_SERVICE_HOST";
+  String PROPERTY_REST_SERVICE_PORT = "REST_SERVICE_PORT";
+  String PROPERTY_APP_LIFETIME = "docker.lifetime";
+  String PROPERTY_APP_RUNAS_USER = "APP_RUNAS_USER";
+  Long DEFAULT_UNLIMITED_LIFETIME = -1l;
+
+  Integer HTTP_STATUS_CODE_ACCEPTED = 202;
+  String ARTIFACT_TYPE_SLIDER_ZIP = "slider-zip";
+
+  Integer GET_APPLICATIONS_THREAD_POOL_SIZE = 200;
+
+  String PROPERTY_PYTHON_PATH = "python.path";
+  String PROPERTY_COMPONENT_TYPE = "site.global.component_type";
+  String PROPERTY_DNS_DEPENDENCY = "site.global.dns.dependency";
+
+  String COMPONENT_TYPE_EXTERNAL = "external";
+
+  String COMMAND_ORDER_SUFFIX_START = "-START";
+  String COMMAND_ORDER_SUFFIX_STARTED = "-STARTED";
+  String EXPORT_GROUP_NAME = "QuickLinks";
+
+  Integer ERROR_CODE_APP_DOES_NOT_EXIST = 404001;
+  Integer ERROR_CODE_APP_IS_NOT_RUNNING = 404002;
+  Integer ERROR_CODE_APP_SUBMITTED_BUT_NOT_RUNNING_YET = 404003;
+  Integer ERROR_CODE_APP_NAME_INVALID = 404004;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiErrorMessages.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiErrorMessages.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiErrorMessages.java
new file mode 100644
index 0000000..685f85a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/utils/RestApiErrorMessages.java
@@ -0,0 +1,79 @@
+/*
+ * 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.services.utils;
+
+public interface RestApiErrorMessages {
+  String ERROR_APPLICATION_NAME_INVALID =
+      "Application name is either empty or not provided";
+  String ERROR_APPLICATION_NAME_INVALID_FORMAT =
+      "Application name is not valid - only lower case letters, digits,"
+          + " underscore and hyphen are allowed";
+
+  String ERROR_APPLICATION_NOT_RUNNING = "Application not running";
+  String ERROR_APPLICATION_DOES_NOT_EXIST = "Application not found";
+
+  String ERROR_SUFFIX_FOR_COMPONENT =
+      " for component %s (nor at the global level)";
+  String ERROR_ARTIFACT_INVALID = "Artifact is not provided";
+  String ERROR_ARTIFACT_FOR_COMP_INVALID =
+      ERROR_ARTIFACT_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_ARTIFACT_ID_INVALID =
+      "Artifact id (like docker image name) is either empty or not provided";
+  String ERROR_ARTIFACT_ID_FOR_COMP_INVALID =
+      ERROR_ARTIFACT_ID_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+
+  String ERROR_RESOURCE_INVALID = "Resource is not provided";
+  String ERROR_RESOURCE_FOR_COMP_INVALID =
+      ERROR_RESOURCE_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_RESOURCE_MEMORY_INVALID =
+      "Application resource or memory not provided";
+  String ERROR_RESOURCE_CPUS_INVALID =
+      "Application resource or cpus not provided";
+  String ERROR_RESOURCE_CPUS_INVALID_RANGE =
+      "Unacceptable no of cpus specified, either zero or negative";
+  String ERROR_RESOURCE_MEMORY_FOR_COMP_INVALID =
+      ERROR_RESOURCE_MEMORY_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_RESOURCE_CPUS_FOR_COMP_INVALID =
+      ERROR_RESOURCE_CPUS_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+  String ERROR_RESOURCE_CPUS_FOR_COMP_INVALID_RANGE =
+      ERROR_RESOURCE_CPUS_INVALID_RANGE
+          + " for component %s (or at the global level)";
+  String ERROR_CONTAINERS_COUNT_INVALID =
+      "Required no of containers not specified";
+  String ERROR_CONTAINERS_COUNT_FOR_COMP_INVALID =
+      ERROR_CONTAINERS_COUNT_INVALID + ERROR_SUFFIX_FOR_COMPONENT;
+
+  String ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_NOT_SUPPORTED =
+      "Cannot specify" + " cpus/memory along with profile";
+  String ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_FOR_COMP_NOT_SUPPORTED =
+      ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_NOT_SUPPORTED
+          + " for component %s";
+  String ERROR_RESOURCE_PROFILE_NOT_SUPPORTED_YET =
+      "Resource profile is not " + "supported yet. Please specify cpus/memory.";
+
+  String ERROR_APPLICATION_IN_USE = "Application name is already in use";
+  String ERROR_NULL_ARTIFACT_ID =
+      "Artifact Id can not be null if artifact type is none";
+  String ERROR_ABSENT_NUM_OF_INSTANCE =
+      "Num of instances should appear either globally or per component";
+  String ERROR_ABSENT_LAUNCH_COMMAND =
+      "launch command should appear if type is slider-zip or none";
+
+  String ERROR_QUICKLINKS_FOR_COMP_INVALID =
+      "Quicklinks specified at component level, needs corresponding values set at application level";
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java
new file mode 100644
index 0000000..b1b6d7c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/webapp/ApplicationApiWebApp.java
@@ -0,0 +1,127 @@
+/*
+ * 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.services.webapp;
+
+import static org.apache.hadoop.yarn.services.utils.RestApiConstants.*;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.URI;
+import java.util.Arrays;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
+import org.mortbay.jetty.webapp.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class launches the web application using Hadoop HttpServer2 (which uses
+ * an embedded Jetty container). This is the entry point to your application.
+ * The Java command used to launch this app should call the main method.
+ */
+public class ApplicationApiWebApp extends AbstractService {
+  private static final Logger logger = LoggerFactory
+      .getLogger(ApplicationApiWebApp.class);
+  private static final String SEP = ";";
+
+  // REST API server for YARN native services
+  private HttpServer2 applicationApiServer;
+
+  public static void main(String[] args) throws IOException {
+    ApplicationApiWebApp apiWebApp = new ApplicationApiWebApp();
+    try {
+      apiWebApp.startWebApp();
+    } catch (Exception e) {
+      if (apiWebApp != null) {
+        apiWebApp.close();
+      }
+    }
+  }
+
+  public ApplicationApiWebApp() {
+    super(ApplicationApiWebApp.class.getName());
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    startWebApp();
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (applicationApiServer != null) {
+      applicationApiServer.stop();
+    }
+    super.serviceStop();
+  }
+
+  protected void startWebApp() throws IOException {
+    // The port that we should run on can be set into an environment variable
+    // Look for that variable and default to 9191 if it isn't there.
+    String webPort = System.getenv(PROPERTY_REST_SERVICE_PORT);
+    if (StringUtils.isEmpty(webPort)) {
+      webPort = "9191";
+    }
+
+    String webHost = System.getenv(PROPERTY_REST_SERVICE_HOST);
+    if (StringUtils.isEmpty(webHost)) {
+      webHost = InetAddress.getLocalHost().getHostName();
+    }
+    logger.info("YARN native services REST API running on host {} and port {}",
+        webHost, webPort);
+    logger.info("Configuration = {}", getConfig());
+
+    applicationApiServer = new HttpServer2.Builder()
+        .setName("services-rest-api")
+        .addEndpoint(URI.create("http://" + webHost + ":" + webPort)).build();
+
+    String apiPackages = "org.apache.hadoop.yarn.services.api" + SEP
+        + "org.apache.hadoop.yarn.services.api.impl" + SEP
+        + "org.apache.hadoop.yarn.services.resource" + SEP
+        + "org.apache.hadoop.yarn.services.utils" + SEP
+        + "org.apache.hadoop.yarn.services.webapp" + SEP
+        + GenericExceptionHandler.class.getPackage().getName() + SEP
+        + YarnJacksonJaxbJsonProvider.class.getPackage().getName();
+    applicationApiServer.addJerseyResourcePackage(apiPackages, CONTEXT_ROOT
+        + "/*");
+
+    try {
+      logger.info("Application starting up. Logging start...");
+      applicationApiServer.start();
+      logger.info("Server status = {}", applicationApiServer.toString());
+      for (Configuration conf : applicationApiServer.getWebAppContext()
+          .getConfigurations()) {
+        logger.info("Configurations = {}", conf);
+      }
+      logger.info("Context Path = {}", Arrays.asList(applicationApiServer
+          .getWebAppContext().getContextPath()));
+      logger.info("ResourceBase = {}", Arrays.asList(applicationApiServer
+          .getWebAppContext().getResourceBase()));
+      logger.info("War = {}",
+          Arrays.asList(applicationApiServer.getWebAppContext().getWar()));
+    } catch (Exception ex) {
+      logger.error("Hadoop HttpServer2 App **failed**", ex);
+      throw ex;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/log4j-server.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/log4j-server.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/log4j-server.properties
new file mode 100644
index 0000000..8c679b9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/log4j-server.properties
@@ -0,0 +1,76 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#       http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+#
+
+# This is the log4j configuration for YARN Services REST API Server
+
+# Log rotation based on size (100KB) with a max of 10 backup files
+log4j.rootLogger=INFO, restservicelog
+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} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n
+
+log4j.appender.restservicelog=org.apache.log4j.RollingFileAppender
+log4j.appender.restservicelog.layout=org.apache.log4j.PatternLayout
+log4j.appender.restservicelog.File=${REST_SERVICE_LOG_DIR}/restservice.log
+log4j.appender.restservicelog.MaxFileSize=1GB
+log4j.appender.restservicelog.MaxBackupIndex=10
+
+# log layout skips stack-trace creation operations by avoiding line numbers and method
+log4j.appender.restservicelog.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} - %m%n
+
+# debug edition is much more expensive
+#log4j.appender.restservicelog.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n
+
+# configure stderr
+# set the conversion pattern of stderr
+# Print the date in ISO 8601 format
+log4j.appender.stderr=org.apache.log4j.ConsoleAppender
+log4j.appender.stderr.Target=System.err
+log4j.appender.stderr.layout=org.apache.log4j.PatternLayout
+log4j.appender.stderr.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} - %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
+
+# for debugging REST API Service
+#log4j.logger.org.apache.hadoop.yarn.services=DEBUG
+
+# uncomment to debug service lifecycle issues
+#log4j.logger.org.apache.hadoop.yarn.service.launcher=DEBUG
+#log4j.logger.org.apache.hadoop.yarn.service=DEBUG
+
+# uncomment for YARN operations
+#log4j.logger.org.apache.hadoop.yarn.client=DEBUG
+
+# uncomment this to debug security problems
+#log4j.logger.org.apache.hadoop.security=DEBUG
+
+#crank back on some noise
+log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
+log4j.logger.org.apache.hadoop.hdfs=WARN
+log4j.logger.org.apache.hadoop.hdfs.shortcircuit=ERROR
+
+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.curator.framework.state=ERROR
+log4j.logger.org.apache.curator.framework.imps=WARN
+
+log4j.logger.org.mortbay.log=DEBUG

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/webapps/services-rest-api/app
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/webapps/services-rest-api/app b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/webapps/services-rest-api/app
new file mode 100644
index 0000000..6a077b1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/resources/webapps/services-rest-api/app
@@ -0,0 +1,16 @@
+# 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.
+
+DON'T DELETE. REST WEBAPP RUN SCRIPT WILL STOP WORKING.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/scripts/run_rest_service.sh
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/scripts/run_rest_service.sh b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/scripts/run_rest_service.sh
new file mode 100644
index 0000000..9f15b7e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/scripts/run_rest_service.sh
@@ -0,0 +1,28 @@
+#!/usr/bin/env bash
+
+# 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.
+
+export SLIDER_VERSION=${project.version}
+export HDP_VERSION=${HDP_VERSION:-$SLIDER_VERSION}
+export SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )"
+export LIB_PARENT_DIR=`dirname $SCRIPT_DIR`
+export JAVA_HOME=${JAVA_HOME:-/usr/jdk64/jdk1.8.0_40}
+export HADOOP_CONF_DIR=${HADOOP_CONF_DIR:-/etc/hadoop/conf}
+export REST_SERVICE_PORT=${REST_SERVICE_PORT:-9191}
+export APP_RUNAS_USER=${APP_RUNAS_USER:-root}
+export REST_SERVICE_LOG_DIR=${REST_SERVICE_LOG_DIR:-/tmp/}
+export JAVA_OPTS="-Xms256m -Xmx1024m -XX:+PrintGC -Xloggc:$REST_SERVICE_LOG_DIR/gc.log"
+$JAVA_HOME/bin/java $JAVA_OPTS -cp .:$HADOOP_CONF_DIR:$LIB_PARENT_DIR/services-api/*:$LIB_PARENT_DIR/slider/* -DREST_SERVICE_LOG_DIR=$REST_SERVICE_LOG_DIR -Dlog4j.configuration=log4j-server.properties -Dslider.libdir=$LIB_PARENT_DIR/slider org.apache.hadoop.yarn.services.webapp.ApplicationApiWebApp 1>>$REST_SERVICE_LOG_DIR/restservice-out.log 2>&1

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/webapp/WEB-INF/web.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/webapp/WEB-INF/web.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/webapp/WEB-INF/web.xml
new file mode 100644
index 0000000..f2f8b5b
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/webapp/WEB-INF/web.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+  http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<web-app xmlns="http://java.sun.com/xml/ns/javaee" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+        xsi:schemaLocation="http://java.sun.com/xml/ns/javaee http://java.sun.com/xml/ns/javaee/web-app_3_0.xsd"
+        version="3.0">
+
+    <servlet>
+        <servlet-name>Jersey REST API</servlet-name>
+        <servlet-class>com.sun.jersey.spi.container.servlet.ServletContainer</servlet-class>
+        <init-param>
+            <param-name>com.sun.jersey.config.property.packages</param-name>
+            <param-value>org.apache.hadoop.yarn.services.webapp,org.apache.hadoop.yarn.services.api,org.apache.hadoop.yarn.services.resource,org.apache.hadoop.yarn.services.api.impl</param-value>
+        </init-param>
+        <init-param>
+          <param-name>com.sun.jersey.api.json.POJOMappingFeature</param-name>
+          <param-value>true</param-value>
+        </init-param>
+        <load-on-startup>1</load-on-startup>
+    </servlet>
+    <servlet-mapping>
+        <servlet-name>Jersey REST API</servlet-name>
+        <url-pattern>/*</url-pattern>
+    </servlet-mapping>
+</web-app>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/services/api/impl/TestApplicationApiService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/services/api/impl/TestApplicationApiService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/services/api/impl/TestApplicationApiService.java
new file mode 100644
index 0000000..a03ab69
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/services/api/impl/TestApplicationApiService.java
@@ -0,0 +1,232 @@
+/*
+ * 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.services.api.impl;
+
+import static org.apache.hadoop.yarn.services.utils.RestApiConstants.*;
+import static org.apache.hadoop.yarn.services.utils.RestApiErrorMessages.*;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.yarn.services.resource.Application;
+import org.apache.hadoop.yarn.services.resource.Artifact;
+import org.apache.hadoop.yarn.services.resource.Resource;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.core.classloader.annotations.SuppressStaticInitializationFor;
+import org.powermock.modules.junit4.PowerMockRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test class for application life time monitor feature test.
+ */
+@RunWith(PowerMockRunner.class)
+@SuppressStaticInitializationFor("org.apache.hadoop.yarn.services.api.impl.ApplicationApiService")
+public class TestApplicationApiService {
+  private static final Logger logger = LoggerFactory
+      .getLogger(TestApplicationApiService.class);
+  private static String EXCEPTION_PREFIX = "Should have thrown exception: ";
+  private static String NO_EXCEPTION_PREFIX = "Should not have thrown exception: ";
+  private ApplicationApiService appApiService;
+
+  @Before
+  public void setup() throws Exception {
+     appApiService = new ApplicationApiService();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+  }
+
+  @Test(timeout = 90000)
+  public void testValidateApplicationPostPayload() throws Exception {
+    Application app = new Application();
+    Map<String, String> compNameArtifactIdMap = new HashMap<>();
+
+    // no name
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX + "application with no name");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_APPLICATION_NAME_INVALID, e.getMessage());
+    }
+
+    // bad format name
+    String[] badNames = { "4finance", "Finance", "finance@home" };
+    for (String badName : badNames) {
+      app.setName(badName);
+      try {
+        appApiService.validateApplicationPostPayload(app,
+            compNameArtifactIdMap);
+        Assert.fail(EXCEPTION_PREFIX + "application with bad name " + badName);
+      } catch (IllegalArgumentException e) {
+        Assert.assertEquals(ERROR_APPLICATION_NAME_INVALID_FORMAT,
+            e.getMessage());
+      }
+    }
+
+    // no artifact
+    app.setName("finance_home");
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX + "application with no artifact");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_ARTIFACT_INVALID, e.getMessage());
+    }
+
+    // no artifact id
+    Artifact artifact = new Artifact();
+    app.setArtifact(artifact);
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX + "application with no artifact id");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_ARTIFACT_ID_INVALID, e.getMessage());
+    }
+
+    // if artifact is of type APPLICATION then everything is valid here
+    artifact.setType(Artifact.TypeEnum.APPLICATION);
+    artifact.setId("app.io/hbase:facebook_0.2");
+    app.setNumberOfContainers(5l);
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+    } catch (IllegalArgumentException e) {
+      logger.error("application attributes specified should be valid here", e);
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+
+    // default-component, default-lifetime and the property component_type
+    // should get assigned here
+    Assert.assertEquals(app.getComponents().get(0).getName(),
+        DEFAULT_COMPONENT_NAME);
+    Assert.assertEquals(app.getLifetime(), DEFAULT_UNLIMITED_LIFETIME);
+    Assert.assertEquals("Property not set",
+        app.getConfiguration().getProperties().get(PROPERTY_COMPONENT_TYPE),
+        COMPONENT_TYPE_EXTERNAL);
+
+    // unset artifact type, default component and no of containers to test other
+    // validation logic
+    artifact.setType(null);
+    app.setComponents(null);
+    app.setNumberOfContainers(null);
+
+    // resource not specified
+    artifact.setId("docker.io/centos:centos7");
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX + "application with no resource");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_RESOURCE_INVALID, e.getMessage());
+    }
+
+    // memory not specified
+    Resource res = new Resource();
+    app.setResource(res);
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX + "application with no memory");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_RESOURCE_MEMORY_INVALID, e.getMessage());
+    }
+
+    // cpus not specified
+    res.setMemory("2gb");
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX + "application with no cpu");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_RESOURCE_CPUS_INVALID, e.getMessage());
+    }
+
+    // invalid no of cpus
+    res.setCpus(-2);
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(
+          EXCEPTION_PREFIX + "application with invalid no of cpups");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_RESOURCE_CPUS_INVALID_RANGE, e.getMessage());
+    }
+
+    // number of containers not specified
+    res.setCpus(2);
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(
+          EXCEPTION_PREFIX + "application with no container count");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_CONTAINERS_COUNT_INVALID, e.getMessage());
+    }
+
+    // specifying profile along with cpus/memory raises exception
+    res.setProfile("hbase_finance_large");
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX
+          + "application with resource profile along with cpus/memory");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_NOT_SUPPORTED,
+          e.getMessage());
+    }
+
+    // currently resource profile alone is not supported.
+    // TODO: remove the next test once it is supported.
+    res.setCpus(null);
+    res.setMemory(null);
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+      Assert.fail(EXCEPTION_PREFIX
+          + "application with resource profile only - NOT SUPPORTED");
+    } catch (IllegalArgumentException e) {
+      Assert.assertEquals(ERROR_RESOURCE_PROFILE_NOT_SUPPORTED_YET,
+          e.getMessage());
+    }
+
+    // unset profile here and add cpus/memory back
+    res.setProfile(null);
+    res.setCpus(2);
+    res.setMemory("2gb");
+
+    // everything valid here
+    app.setNumberOfContainers(5l);
+    try {
+      appApiService.validateApplicationPostPayload(app,
+          compNameArtifactIdMap);
+    } catch (IllegalArgumentException e) {
+      logger.error("application attributes specified should be valid here", e);
+      Assert.fail(NO_EXCEPTION_PREFIX + e.getMessage());
+    }
+
+    // Now test with components
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
index 3496095..bf29856 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
@@ -38,7 +38,7 @@
     <module>hadoop-yarn-applications-distributedshell</module>
     <module>hadoop-yarn-applications-unmanaged-am-launcher</module>
     <module>hadoop-yarn-slider</module>
-
+    <module>hadoop-yarn-services-api</module>
   </modules>
 
  <profiles>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[2/3] hadoop git commit: YARN-5610. Initial code for native services REST API. Contributed by Gour Saha

Posted by ji...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java
new file mode 100644
index 0000000..cfcae95
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Application.java
@@ -0,0 +1,452 @@
+/*
+ * 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.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.slider.providers.PlacementPolicy;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonPropertyOrder;
+
+/**
+ * An Application resource has the following attributes.
+ **/
+
+@ApiModel(description = "An Application resource has the following attributes.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@XmlRootElement
+@JsonInclude(JsonInclude.Include.NON_NULL)
+@JsonPropertyOrder({ " name, state, resource, numberOfContainers, lifetime, containers " })
+public class Application extends BaseResource {
+  private static final long serialVersionUID = -4491694636566094885L;
+
+  private String id = null;
+  private String name = null;
+  private Artifact artifact = null;
+  private Resource resource = null;
+  private String launchCommand = null;
+  private Date launchTime = null;
+  private Long numberOfContainers = null;
+  private Long numberOfRunningContainers = null;
+  private Long lifetime = null;
+  private PlacementPolicy placementPolicy = null;
+  private List<Component> components = null;
+  private Configuration configuration = null;
+  private List<Container> containers = new ArrayList<>();
+  private ApplicationState state = null;
+  private Map<String, String> quicklinks = null;
+  private String queue;
+
+  /**
+   * A unique application id.
+   **/
+  public Application id(String id) {
+    this.id = id;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", required = true, value = "A unique application id.")
+  @JsonProperty("id")
+  public String getId() {
+    return id;
+  }
+
+  public void setId(String id) {
+    this.id = id;
+  }
+
+  /**
+   * A unique application name.
+   **/
+  public Application name(String name) {
+    this.name = name;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", required = true, value = "A unique application name.")
+  @JsonProperty("name")
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  /**
+   * Artifact of single-component applications. Mandatory if components
+   * attribute is not specified.
+   **/
+  public Application artifact(Artifact artifact) {
+    this.artifact = artifact;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Artifact of single-component applications. Mandatory if components attribute is not specified.")
+  @JsonProperty("artifact")
+  public Artifact getArtifact() {
+    return artifact;
+  }
+
+  public void setArtifact(Artifact artifact) {
+    this.artifact = artifact;
+  }
+
+  /**
+   * Resource of single-component applications or the global default for
+   * multi-component applications. Mandatory if it is a single-component
+   * application and if cpus and memory are not specified at the Application
+   * level.
+   **/
+  public Application resource(Resource resource) {
+    this.resource = resource;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Resource of single-component applications or the global default for multi-component applications. Mandatory if it is a single-component application and if cpus and memory are not specified at the Application level.")
+  @JsonProperty("resource")
+  public Resource getResource() {
+    return resource;
+  }
+
+  public void setResource(Resource resource) {
+    this.resource = resource;
+  }
+
+  /**
+   * The custom launch command of an application component (optional). If not
+   * specified for applications with docker images say, it will default to the
+   * default start command of the image. If there is a single component in this
+   * application, you can specify this without the need to have a 'components'
+   * section.
+   **/
+  public Application launchCommand(String launchCommand) {
+    this.launchCommand = launchCommand;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The custom launch command of an application component (optional). If not specified for applications with docker images say, it will default to the default start command of the image. If there is a single component in this application, you can specify this without the need to have a 'components' section.")
+  @JsonProperty("launch_command")
+  public String getLaunchCommand() {
+    return launchCommand;
+  }
+
+  @XmlElement(name = "launch_command")
+  public void setLaunchCommand(String launchCommand) {
+    this.launchCommand = launchCommand;
+  }
+
+  /**
+   * The time when the application was created, e.g. 2016-03-16T01:01:49.000Z.
+   **/
+  public Application launchTime(Date launchTime) {
+    this.launchTime = launchTime;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The time when the application was created, e.g. 2016-03-16T01:01:49.000Z.")
+  @JsonProperty("launch_time")
+  public String getLaunchTime() {
+    return launchTime.toString();
+  }
+
+  @XmlElement(name = "launch_time")
+  public void setLaunchTime(Date launchTime) {
+    this.launchTime = launchTime;
+  }
+
+  /**
+   * Number of containers for each app-component in the application. Each
+   * app-component can further override this app-level global default.
+   **/
+  public Application numberOfContainers(Long numberOfContainers) {
+    this.numberOfContainers = numberOfContainers;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Number of containers for each app-component in the application. Each app-component can further override this app-level global default.")
+  @JsonProperty("number_of_containers")
+  public Long getNumberOfContainers() {
+    return numberOfContainers;
+  }
+
+  @XmlElement(name = "number_of_containers")
+  public void setNumberOfContainers(Long numberOfContainers) {
+    this.numberOfContainers = numberOfContainers;
+  }
+
+  /**
+   * In get response this provides the total number of running containers for
+   * this application (across all components) at the time of request. Note, a
+   * subsequent request can return a different number as and when more
+   * containers get allocated until it reaches the total number of containers or
+   * if a flex request has been made between the two requests.
+   **/
+  public Application numberOfRunningContainers(Long numberOfRunningContainers) {
+    this.numberOfRunningContainers = numberOfRunningContainers;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "In get response this provides the total number of running containers for this application (across all components) at the time of request. Note, a subsequent request can return a different number as and when more containers get allocated until it reaches the total number of containers or if a flex request has been made between the two requests.")
+  @JsonProperty("number_of_running_containers")
+  public Long getNumberOfRunningContainers() {
+    return numberOfRunningContainers;
+  }
+
+  @XmlElement(name = "number_of_running_containers")
+  public void setNumberOfRunningContainers(Long numberOfRunningContainers) {
+    this.numberOfRunningContainers = numberOfRunningContainers;
+  }
+
+  /**
+   * Life time (in seconds) of the application from the time it reaches the
+   * STARTED state (after which it is automatically destroyed by YARN). For
+   * unlimited lifetime do not set a lifetime value.
+   **/
+  public Application lifetime(Long lifetime) {
+    this.lifetime = lifetime;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Life time (in seconds) of the application from the time it reaches the STARTED state (after which it is automatically destroyed by YARN). For unlimited lifetime do not set a lifetime value.")
+  @JsonProperty("lifetime")
+  public Long getLifetime() {
+    return lifetime;
+  }
+
+  public void setLifetime(Long lifetime) {
+    this.lifetime = lifetime;
+  }
+
+  /**
+   * Advanced scheduling and placement policies (optional). If not specified, it
+   * defaults to the default placement policy of the app owner. The design of
+   * placement policies are in the works. It is not very clear at this point,
+   * how policies in conjunction with labels be exposed to application owners.
+   * This is a placeholder for now. The advanced structure of this attribute
+   * will be determined by YARN-4902.
+   **/
+  public Application placementPolicy(PlacementPolicy placementPolicy) {
+    this.placementPolicy = placementPolicy;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Advanced scheduling and placement policies (optional). If not specified, it defaults to the default placement policy of the app owner. The design of placement policies are in the works. It is not very clear at this point, how policies in conjunction with labels be exposed to application owners. This is a placeholder for now. The advanced structure of this attribute will be determined by YARN-4902.")
+  @JsonProperty("placement_policy")
+  public PlacementPolicy getPlacementPolicy() {
+    return placementPolicy;
+  }
+
+  @XmlElement(name = "placement_policy")
+  public void setPlacementPolicy(PlacementPolicy placementPolicy) {
+    this.placementPolicy = placementPolicy;
+  }
+
+  /**
+   * Components of an application.
+   **/
+  public Application components(List<Component> components) {
+    this.components = components;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Components of an application.")
+  @JsonProperty("components")
+  public List<Component> getComponents() {
+    return components;
+  }
+
+  public void setComponents(List<Component> components) {
+    this.components = components;
+  }
+
+  /**
+   * Config properties of an application. Configurations provided at the
+   * application/global level are available to all the components. Specific
+   * properties can be overridden at the component level.
+   **/
+  public Application configuration(Configuration configuration) {
+    this.configuration = configuration;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Config properties of an application. Configurations provided at the application/global level are available to all the components. Specific properties can be overridden at the component level.")
+  @JsonProperty("configuration")
+  public Configuration getConfiguration() {
+    return configuration;
+  }
+
+  public void setConfiguration(Configuration configuration) {
+    this.configuration = configuration;
+  }
+
+  /**
+   * Containers of a started application. Specifying a value for this attribute
+   * for the POST payload raises a validation error. This blob is available only
+   * in the GET response of a started application.
+   **/
+  public Application containers(List<Container> containers) {
+    this.containers = containers;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Containers of a started application. Specifying a value for this attribute for the POST payload raises a validation error. This blob is available only in the GET response of a started application.")
+  @JsonProperty("containers")
+  public List<Container> getContainers() {
+    return containers;
+  }
+
+  public void setContainers(List<Container> containers) {
+    this.containers = containers;
+  }
+
+  public void addContainer(Container container) {
+    this.containers.add(container);
+  }
+
+  /**
+   * State of the application. Specifying a value for this attribute for the
+   * POST payload raises a validation error. This attribute is available only in
+   * the GET response of a started application.
+   **/
+  public Application state(ApplicationState state) {
+    this.state = state;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "State of the application. Specifying a value for this attribute for the POST payload raises a validation error. This attribute is available only in the GET response of a started application.")
+  @JsonProperty("state")
+  public ApplicationState getState() {
+    return state;
+  }
+
+  public void setState(ApplicationState state) {
+    this.state = state;
+  }
+
+  /**
+   * A blob of key-value pairs of quicklinks to be exported for an application.
+   **/
+  public Application quicklinks(Map<String, String> quicklinks) {
+    this.quicklinks = quicklinks;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A blob of key-value pairs of quicklinks to be exported for an application.")
+  @JsonProperty("quicklinks")
+  public Map<String, String> getQuicklinks() {
+    return quicklinks;
+  }
+
+  public void setQuicklinks(Map<String, String> quicklinks) {
+    this.quicklinks = quicklinks;
+  }
+
+  /**
+   * The YARN queue that this application should be submitted to.
+   **/
+  public Application queue(String queue) {
+    this.queue = queue;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The YARN queue that this application should be submitted to.")
+  @JsonProperty("queue")
+  public String getQueue() {
+    return queue;
+  }
+
+  public void setQueue(String queue) {
+    this.queue = queue;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Application application = (Application) o;
+    return Objects.equals(this.name, application.name);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(name);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Application {\n");
+
+    sb.append("    name: ").append(toIndentedString(name)).append("\n");
+    sb.append("    id: ").append(toIndentedString(id)).append("\n");
+    sb.append("    artifact: ").append(toIndentedString(artifact)).append("\n");
+    sb.append("    resource: ").append(toIndentedString(resource)).append("\n");
+    sb.append("    launchCommand: ").append(toIndentedString(launchCommand))
+        .append("\n");
+    sb.append("    launchTime: ").append(toIndentedString(launchTime))
+        .append("\n");
+    sb.append("    numberOfContainers: ")
+        .append(toIndentedString(numberOfContainers)).append("\n");
+    sb.append("    numberOfRunningContainers: ")
+        .append(toIndentedString(numberOfRunningContainers)).append("\n");
+    sb.append("    lifetime: ").append(toIndentedString(lifetime)).append("\n");
+    sb.append("    placementPolicy: ")
+        .append(toIndentedString(placementPolicy)).append("\n");
+    sb.append("    components: ").append(toIndentedString(components))
+        .append("\n");
+    sb.append("    configuration: ").append(toIndentedString(configuration))
+        .append("\n");
+    sb.append("    containers: ").append(toIndentedString(containers))
+        .append("\n");
+    sb.append("    state: ").append(toIndentedString(state)).append("\n");
+    sb.append("    quicklinks: ").append(toIndentedString(quicklinks))
+        .append("\n");
+    sb.append("    queue: ").append(toIndentedString(queue)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationState.java
new file mode 100644
index 0000000..ae96e8a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationState.java
@@ -0,0 +1,25 @@
+/*
+ * 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.services.resource;
+
+/**
+ * The current state of an application.
+ **/
+public enum ApplicationState {
+  ACCEPTED, STARTED, READY, STOPPED, FAILED;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationStatus.java
new file mode 100644
index 0000000..0166b48
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ApplicationStatus.java
@@ -0,0 +1,147 @@
+/*
+ * 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.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * The current status of a submitted application, returned as a response to the
+ * GET API.
+ **/
+
+@ApiModel(description = "The current status of a submitted application, returned as a response to the GET api")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@XmlRootElement
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class ApplicationStatus extends BaseResource {
+  private static final long serialVersionUID = -3469885905347851034L;
+
+  private String diagnostics = null;
+  private ApplicationState state = null;
+  private Integer code = null;
+
+  /**
+   * Diagnostic information (if any) for the reason of the current state of the
+   * application. It typically has a non-null value, if the application is in a
+   * non-running state.
+   **/
+  public ApplicationStatus diagnostics(String diagnostics) {
+    this.diagnostics = diagnostics;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Diagnostic information (if any) for the reason of the current state of the application. It typically has a non-null value, if the application is in a non-running state.")
+  @JsonProperty("diagnostics")
+  public String getDiagnostics() {
+    return diagnostics;
+  }
+
+  @XmlElement(name = "diagnostics")
+  public void setDiagnostics(String diagnostics) {
+    this.diagnostics = diagnostics;
+  }
+
+  /**
+   * Application state
+   **/
+  public ApplicationStatus state(ApplicationState state) {
+    this.state = state;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Application state")
+  @JsonProperty("state")
+  public ApplicationState getState() {
+    return state;
+  }
+
+  public void setState(ApplicationState state) {
+    this.state = state;
+  }
+
+  /**
+   * An error code specific to a scenario which app owners should be able to use
+   * to understand the failure in addition to the diagnostic information.
+   **/
+  public ApplicationStatus code(Integer code) {
+    this.code = code;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "An error code specific to a scenario which app owners should be able to use to understand the failure in addition to the diagnostic information.")
+  @JsonProperty("code")
+  public Integer getCode() {
+    return code;
+  }
+
+  public void setCode(Integer code) {
+    this.code = code;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ApplicationStatus applicationStatus = (ApplicationStatus) o;
+    return Objects.equals(this.diagnostics, applicationStatus.diagnostics)
+        && Objects.equals(this.state, applicationStatus.state)
+        && Objects.equals(this.code, applicationStatus.code);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(diagnostics, state, code);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class ApplicationStatus {\n");
+
+    sb.append("    diagnostics: ").append(toIndentedString(diagnostics))
+        .append("\n");
+    sb.append("    state: ").append(toIndentedString(state)).append("\n");
+    sb.append("    code: ").append(toIndentedString(code)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
new file mode 100644
index 0000000..aee4d11
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Artifact.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonValue;
+
+/**
+ * Artifact of an application component.
+ **/
+
+@ApiModel(description = "Artifact of an application component")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class Artifact {
+
+  private String id = null;
+
+  public enum TypeEnum {
+    DOCKER("docker"), TARBALL("tarball"), APPLICATION("application");
+
+    private String value;
+
+    TypeEnum(String value) {
+      this.value = value;
+    }
+
+    @Override
+    @JsonValue
+    public String toString() {
+      return value;
+    }
+  }
+
+  private TypeEnum type = TypeEnum.DOCKER;
+  private String uri = null;
+
+  /**
+   * Artifact id. Examples are package location uri for tarball based apps,
+   * image name for docker, etc.
+   **/
+  public Artifact id(String id) {
+    this.id = id;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", required = true, value = "Artifact id. Examples are package location uri for tarball based apps, image name for docker, etc.")
+  @JsonProperty("id")
+  public String getId() {
+    return id;
+  }
+
+  public void setId(String id) {
+    this.id = id;
+  }
+
+  /**
+   * Artifact type, like docker, tarball, etc. (optional)
+   **/
+  public Artifact type(TypeEnum type) {
+    this.type = type;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Artifact type, like docker, tarball, etc. (optional)")
+  @JsonProperty("type")
+  public TypeEnum getType() {
+    return type;
+  }
+
+  public void setType(TypeEnum type) {
+    this.type = type;
+  }
+
+  /**
+   * Artifact location to support multiple artifact stores (optional).
+   **/
+  public Artifact uri(String uri) {
+    this.uri = uri;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Artifact location to support multiple artifact stores (optional).")
+  @JsonProperty("uri")
+  public String getUri() {
+    return uri;
+  }
+
+  public void setUri(String uri) {
+    this.uri = uri;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Artifact artifact = (Artifact) o;
+    return Objects.equals(this.id, artifact.id)
+        && Objects.equals(this.type, artifact.type)
+        && Objects.equals(this.uri, artifact.uri);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(id, type, uri);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Artifact {\n");
+
+    sb.append("    id: ").append(toIndentedString(id)).append("\n");
+    sb.append("    type: ").append(toIndentedString(type)).append("\n");
+    sb.append("    uri: ").append(toIndentedString(uri)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/BaseResource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/BaseResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/BaseResource.java
new file mode 100644
index 0000000..3b2c8b1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/BaseResource.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import java.io.Serializable;
+
+public class BaseResource implements Serializable {
+  private static final long serialVersionUID = 1492603053176889431L;
+
+  private String uri;
+
+  /**
+   * Resource location, e.g. \
+   * "/applications/helloworld/containers/container_e3751_1458061340047_0008_01_000002\
+   * "
+   **/
+  public String getUri() {
+    return uri;
+  }
+
+  public void setUri(String uri) {
+    this.uri = uri;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder();
+    builder.append("BaseResource [uri=");
+    builder.append(uri);
+    builder.append("]");
+    return builder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java
new file mode 100644
index 0000000..3ff6945
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Component.java
@@ -0,0 +1,377 @@
+/*
+ * 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.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * One or more components of the application. If the application is HBase say,
+ * then the component can be a simple role like master or regionserver. If the
+ * application is a complex business webapp then a component can be other
+ * applications say Kafka or Storm. Thereby it opens up the support for complex
+ * and nested applications.
+ **/
+
+@ApiModel(description = "One or more components of the application. If the application is HBase say, then the component can be a simple role like master or regionserver. If the application is a complex business webapp then a component can be other applications say Kafka or Storm. Thereby it opens up the support for complex and nested applications.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@XmlRootElement
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class Component {
+
+  private String name = null;
+  private List<String> dependencies = new ArrayList<String>();
+  private ReadinessCheck readinessCheck = null;
+  private Artifact artifact = null;
+  private String launchCommand = null;
+  private Resource resource = null;
+  private Long numberOfContainers = null;
+  private Boolean uniqueComponentSupport = null;
+  private Boolean runPrivilegedContainer = null;
+  private PlacementPolicy placementPolicy = null;
+  private Configuration configuration = null;
+  private List<String> quicklinks = new ArrayList<String>();
+
+  /**
+   * Name of the application component (mandatory).
+   **/
+  public Component name(String name) {
+    this.name = name;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", required = true, value = "Name of the application component (mandatory).")
+  @JsonProperty("name")
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  /**
+   * An array of application components which should be in READY state (as
+   * defined by readiness check), before this component can be started. The
+   * dependencies across all components of an application should be represented
+   * as a DAG.
+   **/
+  public Component dependencies(List<String> dependencies) {
+    this.dependencies = dependencies;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "An array of application components which should be in READY state (as defined by readiness check), before this component can be started. The dependencies across all components of an application should be represented as a DAG.")
+  @JsonProperty("dependencies")
+  public List<String> getDependencies() {
+    return dependencies;
+  }
+
+  public void setDependencies(List<String> dependencies) {
+    this.dependencies = dependencies;
+  }
+
+  /**
+   * Readiness check for this app-component.
+   **/
+  public Component readinessCheck(ReadinessCheck readinessCheck) {
+    this.readinessCheck = readinessCheck;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Readiness check for this app-component.")
+  @JsonProperty("readiness_check")
+  public ReadinessCheck getReadinessCheck() {
+    return readinessCheck;
+  }
+
+  @XmlElement(name = "readiness_check")
+  public void setReadinessCheck(ReadinessCheck readinessCheck) {
+    this.readinessCheck = readinessCheck;
+  }
+
+  /**
+   * Artifact of the component (optional). If not specified, the application
+   * level global artifact takes effect.
+   **/
+  public Component artifact(Artifact artifact) {
+    this.artifact = artifact;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Artifact of the component (optional). If not specified, the application level global artifact takes effect.")
+  @JsonProperty("artifact")
+  public Artifact getArtifact() {
+    return artifact;
+  }
+
+  public void setArtifact(Artifact artifact) {
+    this.artifact = artifact;
+  }
+
+  /**
+   * The custom launch command of this component (optional). When specified at
+   * the component level, it overrides the value specified at the global level
+   * (if any).
+   **/
+  public Component launchCommand(String launchCommand) {
+    this.launchCommand = launchCommand;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The custom launch command of this component (optional). When specified at the component level, it overrides the value specified at the global level (if any).")
+  @JsonProperty("launch_command")
+  public String getLaunchCommand() {
+    return launchCommand;
+  }
+
+  @XmlElement(name = "launch_command")
+  public void setLaunchCommand(String launchCommand) {
+    this.launchCommand = launchCommand;
+  }
+
+  /**
+   * Resource of this component (optional). If not specified, the application
+   * level global resource takes effect.
+   **/
+  public Component resource(Resource resource) {
+    this.resource = resource;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Resource of this component (optional). If not specified, the application level global resource takes effect.")
+  @JsonProperty("resource")
+  public Resource getResource() {
+    return resource;
+  }
+
+  public void setResource(Resource resource) {
+    this.resource = resource;
+  }
+
+  /**
+   * Number of containers for this app-component (optional). If not specified,
+   * the application level global number_of_containers takes effect.
+   **/
+  public Component numberOfContainers(Long numberOfContainers) {
+    this.numberOfContainers = numberOfContainers;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Number of containers for this app-component (optional). If not specified, the application level global number_of_containers takes effect.")
+  @JsonProperty("number_of_containers")
+  public Long getNumberOfContainers() {
+    return numberOfContainers;
+  }
+
+  @XmlElement(name = "number_of_containers")
+  public void setNumberOfContainers(Long numberOfContainers) {
+    this.numberOfContainers = numberOfContainers;
+  }
+
+  /**
+   * Certain applications need to define multiple components using the same
+   * artifact and resource profile, differing only in configurations. In such
+   * cases, this field helps app owners to avoid creating multiple component
+   * definitions with repeated information. The number_of_containers field
+   * dictates the initial number of components created. Component names
+   * typically differ with a trailing id, but assumptions should not be made on
+   * that, as the algorithm can change at any time. Configurations section will
+   * be able to use placeholders like ${APP_COMPONENT_NAME} to get its component
+   * name at runtime, and thereby differing in value at runtime. The best part
+   * of this feature is that when the component is flexed up, entirely new
+   * components (with new trailing ids) are created.
+   **/
+  public Component uniqueComponentSupport(Boolean uniqueComponentSupport) {
+    this.uniqueComponentSupport = uniqueComponentSupport;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Certain applications need to define multiple components using the same artifact and resource profile, differing only in configurations. In such cases, this field helps app owners to avoid creating multiple component definitions with repeated information. The number_of_containers field dictates the initial number of components created. Component names typically differ with a trailing id, but assumptions should not be made on that, as the algorithm can change at any time. Configurations section will be able to use placeholders like ${APP_COMPONENT_NAME} to get its component name at runtime, and thereby differing in value at runtime. The best part of this feature is that when the component is flexed up, entirely new components (with new trailing ids) are created.")
+  @JsonProperty("unique_component_support")
+  public Boolean getUniqueComponentSupport() {
+    return uniqueComponentSupport;
+  }
+
+  @XmlElement(name = "unique_component_support")
+  public void setUniqueComponentSupport(Boolean uniqueComponentSupport) {
+    this.uniqueComponentSupport = uniqueComponentSupport;
+  }
+
+  /**
+   * Run all containers of this component in privileged mode (YARN-4262).
+   **/
+  public Component runPrivilegedContainer(Boolean runPrivilegedContainer) {
+    this.runPrivilegedContainer = runPrivilegedContainer;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Run all containers of this component in privileged mode (YARN-4262).")
+  @JsonProperty("run_privileged_container")
+  public Boolean getRunPrivilegedContainer() {
+    return runPrivilegedContainer;
+  }
+
+  @XmlElement(name = "run_privileged_container")
+  public void setRunPrivilegedContainer(Boolean runPrivilegedContainer) {
+    this.runPrivilegedContainer = runPrivilegedContainer;
+  }
+
+  /**
+   * Advanced scheduling and placement policies for all containers of this
+   * component (optional). If not specified, the app level placement_policy
+   * takes effect. Refer to the description at the global level for more
+   * details.
+   **/
+  public Component placementPolicy(PlacementPolicy placementPolicy) {
+    this.placementPolicy = placementPolicy;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Advanced scheduling and placement policies for all containers of this component (optional). If not specified, the app level placement_policy takes effect. Refer to the description at the global level for more details.")
+  @JsonProperty("placement_policy")
+  public PlacementPolicy getPlacementPolicy() {
+    return placementPolicy;
+  }
+
+  @XmlElement(name = "placement_policy")
+  public void setPlacementPolicy(PlacementPolicy placementPolicy) {
+    this.placementPolicy = placementPolicy;
+  }
+
+  /**
+   * Config properties for this app-component.
+   **/
+  public Component configuration(Configuration configuration) {
+    this.configuration = configuration;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Config properties for this app-component.")
+  @JsonProperty("configuration")
+  public Configuration getConfiguration() {
+    return configuration;
+  }
+
+  public void setConfiguration(Configuration configuration) {
+    this.configuration = configuration;
+  }
+
+  /**
+   * A list of quicklink keys defined at the application level, and to be
+   * resolved by this component.
+   **/
+  public Component quicklinks(List<String> quicklinks) {
+    this.quicklinks = quicklinks;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A list of quicklink keys defined at the application level, and to be resolved by this component.")
+  @JsonProperty("quicklinks")
+  public List<String> getQuicklinks() {
+    return quicklinks;
+  }
+
+  public void setQuicklinks(List<String> quicklinks) {
+    this.quicklinks = quicklinks;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Component component = (Component) o;
+    return Objects.equals(this.name, component.name)
+        && Objects.equals(this.dependencies, component.dependencies)
+        && Objects.equals(this.readinessCheck, component.readinessCheck)
+        && Objects.equals(this.artifact, component.artifact)
+        && Objects.equals(this.launchCommand, component.launchCommand)
+        && Objects.equals(this.resource, component.resource)
+        && Objects
+            .equals(this.numberOfContainers, component.numberOfContainers)
+        && Objects.equals(this.uniqueComponentSupport,
+            component.uniqueComponentSupport)
+        && Objects.equals(this.runPrivilegedContainer,
+            component.runPrivilegedContainer)
+        && Objects.equals(this.placementPolicy, component.placementPolicy)
+        && Objects.equals(this.configuration, component.configuration)
+        && Objects.equals(this.quicklinks, component.quicklinks);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(name, dependencies, readinessCheck, artifact,
+        launchCommand, resource, numberOfContainers, uniqueComponentSupport,
+        runPrivilegedContainer, placementPolicy, configuration, quicklinks);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Component {\n");
+
+    sb.append("    name: ").append(toIndentedString(name)).append("\n");
+    sb.append("    dependencies: ").append(toIndentedString(dependencies))
+        .append("\n");
+    sb.append("    readinessCheck: ").append(toIndentedString(readinessCheck))
+        .append("\n");
+    sb.append("    artifact: ").append(toIndentedString(artifact)).append("\n");
+    sb.append("    launchCommand: ").append(toIndentedString(launchCommand))
+        .append("\n");
+    sb.append("    resource: ").append(toIndentedString(resource)).append("\n");
+    sb.append("    numberOfContainers: ")
+        .append(toIndentedString(numberOfContainers)).append("\n");
+    sb.append("    uniqueComponentSupport: ")
+        .append(toIndentedString(uniqueComponentSupport)).append("\n");
+    sb.append("    runPrivilegedContainer: ")
+        .append(toIndentedString(runPrivilegedContainer)).append("\n");
+    sb.append("    placementPolicy: ")
+        .append(toIndentedString(placementPolicy)).append("\n");
+    sb.append("    configuration: ").append(toIndentedString(configuration))
+        .append("\n");
+    sb.append("    quicklinks: ").append(toIndentedString(quicklinks))
+        .append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java
new file mode 100644
index 0000000..d06c1b8
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ConfigFile.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonValue;
+
+/**
+ * A config file that needs to be created and made available as a volume in an
+ * application component container.
+ **/
+
+@ApiModel(description = "A config file that needs to be created and made available as a volume in an application component container.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@XmlRootElement
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class ConfigFile {
+
+  public enum TypeEnum {
+    XML("xml"), PROPERTIES("properties"), JSON("json"), YAML("yaml"), TEMPLATE(
+        "template"), ENV("env"), HADOOP_XML("hadoop_xml");
+
+    private String value;
+
+    TypeEnum(String value) {
+      this.value = value;
+    }
+
+    @Override
+    @JsonValue
+    public String toString() {
+      return value;
+    }
+  }
+
+  private TypeEnum type = null;
+  private String destFile = null;
+  private String srcFile = null;
+  private Object props = null;
+
+  /**
+   * Config file in the standard format like xml, properties, json, yaml,
+   * template.
+   **/
+  public ConfigFile type(TypeEnum type) {
+    this.type = type;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Config file in the standard format like xml, properties, json, yaml, template.")
+  @JsonProperty("type")
+  public TypeEnum getType() {
+    return type;
+  }
+
+  public void setType(TypeEnum type) {
+    this.type = type;
+  }
+
+  /**
+   * The absolute path that this configuration file should be mounted as, in the
+   * application container.
+   **/
+  public ConfigFile destFile(String destFile) {
+    this.destFile = destFile;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The absolute path that this configuration file should be mounted as, in the application container.")
+  @JsonProperty("dest_file")
+  public String getDestFile() {
+    return destFile;
+  }
+
+  @XmlElement(name = "dest_file")
+  public void setDestFile(String destFile) {
+    this.destFile = destFile;
+  }
+
+  /**
+   * Required for type template. This provides the source location of the
+   * template which needs to be mounted as dest_file post property
+   * substitutions. Typically the src_file would point to a source controlled
+   * network accessible file maintained by tools like puppet, chef, etc.
+   **/
+  public ConfigFile srcFile(String srcFile) {
+    this.srcFile = srcFile;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Required for type template. This provides the source location of the template which needs to be mounted as dest_file post property substitutions. Typically the src_file would point to a source controlled network accessible file maintained by tools like puppet, chef, etc.")
+  @JsonProperty("src_file")
+  public String getSrcFile() {
+    return srcFile;
+  }
+
+  @XmlElement(name = "src_file")
+  public void setSrcFile(String srcFile) {
+    this.srcFile = srcFile;
+  }
+
+  /**
+   * A blob of key value pairs that will be dumped in the dest_file in the
+   * format as specified in type. If the type is template then the attribute
+   * src_file is mandatory and the src_file content is dumped to dest_file post
+   * property substitutions.
+   **/
+  public ConfigFile props(Object props) {
+    this.props = props;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A blob of key value pairs that will be dumped in the dest_file in the format as specified in type. If the type is template then the attribute src_file is mandatory and the src_file content is dumped to dest_file post property substitutions.")
+  @JsonProperty("props")
+  public Object getProps() {
+    return props;
+  }
+
+  public void setProps(Object props) {
+    this.props = props;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ConfigFile configFile = (ConfigFile) o;
+    return Objects.equals(this.type, configFile.type)
+        && Objects.equals(this.destFile, configFile.destFile)
+        && Objects.equals(this.srcFile, configFile.srcFile)
+        && Objects.equals(this.props, configFile.props);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(type, destFile, srcFile, props);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class ConfigFile {\n");
+
+    sb.append("    type: ").append(toIndentedString(type)).append("\n");
+    sb.append("    destFile: ").append(toIndentedString(destFile)).append("\n");
+    sb.append("    srcFile: ").append(toIndentedString(srcFile)).append("\n");
+    sb.append("    props: ").append(toIndentedString(props)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Configuration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Configuration.java
new file mode 100644
index 0000000..05983db
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Configuration.java
@@ -0,0 +1,147 @@
+/*
+ * 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.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Set of configuration properties that can be injected into the application
+ * components via envs, files and custom pluggable helper docker containers.
+ * Files of several standard formats like xml, properties, json, yaml and
+ * templates will be supported.
+ **/
+
+@ApiModel(description = "Set of configuration properties that can be injected into the application components via envs, files and custom pluggable helper docker containers. Files of several standard formats like xml, properties, json, yaml and templates will be supported.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class Configuration {
+
+  private Map<String, String> properties = new HashMap<String, String>();
+  private Map<String, String> env = new HashMap<String, String>();
+  private List<ConfigFile> files = new ArrayList<ConfigFile>();
+
+  /**
+   * A blob of key-value pairs of common application properties.
+   **/
+  public Configuration properties(Map<String, String> properties) {
+    this.properties = properties;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A blob of key-value pairs of common application properties.")
+  @JsonProperty("properties")
+  public Map<String, String> getProperties() {
+    return properties;
+  }
+
+  public void setProperties(Map<String, String> properties) {
+    this.properties = properties;
+  }
+
+  /**
+   * A blob of key-value pairs which will be appended to the default system
+   * properties and handed off to the application at start time. All placeholder
+   * references to properties will be substituted before injection.
+   **/
+  public Configuration env(Map<String, String> env) {
+    this.env = env;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A blob of key-value pairs which will be appended to the default system properties and handed off to the application at start time. All placeholder references to properties will be substituted before injection.")
+  @JsonProperty("env")
+  public Map<String, String> getEnv() {
+    return env;
+  }
+
+  public void setEnv(Map<String, String> env) {
+    this.env = env;
+  }
+
+  /**
+   * Array of list of files that needs to be created and made available as
+   * volumes in the application component containers.
+   **/
+  public Configuration files(List<ConfigFile> files) {
+    this.files = files;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Array of list of files that needs to be created and made available as volumes in the application component containers.")
+  @JsonProperty("files")
+  public List<ConfigFile> getFiles() {
+    return files;
+  }
+
+  public void setFiles(List<ConfigFile> files) {
+    this.files = files;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Configuration configuration = (Configuration) o;
+    return Objects.equals(this.properties, configuration.properties)
+        && Objects.equals(this.env, configuration.env)
+        && Objects.equals(this.files, configuration.files);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(properties, env, files);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Configuration {\n");
+
+    sb.append("    properties: ").append(toIndentedString(properties))
+        .append("\n");
+    sb.append("    env: ").append(toIndentedString(env)).append("\n");
+    sb.append("    files: ").append(toIndentedString(files)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java
new file mode 100644
index 0000000..2faf6f2
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Container.java
@@ -0,0 +1,256 @@
+/*
+ * 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.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Date;
+import java.util.Objects;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+@ApiModel(description = "An instance of a running application container")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@XmlRootElement
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class Container extends BaseResource {
+  private static final long serialVersionUID = -8955788064529288L;
+
+  private String id = null;
+  private Date launchTime = null;
+  private String ip = null;
+  private String hostname = null;
+  private String bareHost = null;
+  private ContainerState state = null;
+  private String componentName = null;
+  private Resource resource = null;
+
+  /**
+   * Unique container id of a running application, e.g.
+   * container_e3751_1458061340047_0008_01_000002
+   **/
+  public Container id(String id) {
+    this.id = id;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Unique container id of a running application, e.g. container_e3751_1458061340047_0008_01_000002")
+  @JsonProperty("id")
+  public String getId() {
+    return id;
+  }
+
+  public void setId(String id) {
+    this.id = id;
+  }
+
+  /**
+   * The time when the container was created, e.g. 2016-03-16T01:01:49.000Z. This will most likely be different from cluster launch time.
+   **/
+  public Container launchTime(Date launchTime) {
+    this.launchTime = launchTime;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The time when the container was created, e.g. 2016-03-16T01:01:49.000Z. This will most likely be different from cluster launch time.")
+  @JsonProperty("launch_time")
+  public String getLaunchTime() {
+    return launchTime.toString();
+  }
+
+  @XmlElement(name = "launch_time")
+  public void setLaunchTime(Date launchTime) {
+    this.launchTime = launchTime;
+  }
+
+  /**
+   * IP address of a running container, e.g. 172.31.42.141. The IP address and
+   * hostname attribute values are dependent on the cluster/docker network setup
+   * as per YARN-4007.
+   **/
+  public Container ip(String ip) {
+    this.ip = ip;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "IP address of a running container, e.g. 172.31.42.141. The IP address and hostname attribute values are dependent on the cluster/docker network setup as per YARN-4007.")
+  @JsonProperty("ip")
+  public String getIp() {
+    return ip;
+  }
+
+  public void setIp(String ip) {
+    this.ip = ip;
+  }
+
+  /**
+   * Fully qualified hostname of a running container, e.g.
+   * ctr-e3751-1458061340047-0008-01-000002.examplestg.site. The IP address and
+   * hostname attribute values are dependent on the cluster/docker network setup
+   * as per YARN-4007.
+   **/
+  public Container hostname(String hostname) {
+    this.hostname = hostname;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Fully qualified hostname of a running container, e.g. ctr-e3751-1458061340047-0008-01-000002.examplestg.site. The IP address and hostname attribute values are dependent on the cluster/docker network setup as per YARN-4007.")
+  @JsonProperty("hostname")
+  public String getHostname() {
+    return hostname;
+  }
+
+  public void setHostname(String hostname) {
+    this.hostname = hostname;
+  }
+
+  /**
+   * The bare node or host in which the container is running, e.g.
+   * cn008.example.com
+   **/
+  public Container bareHost(String bareHost) {
+    this.bareHost = bareHost;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The bare node or host in which the container is running, e.g. cn008.example.com")
+  @JsonProperty("bare_host")
+  public String getBareHost() {
+    return bareHost;
+  }
+
+  @XmlElement(name = "bare_host")
+  public void setBareHost(String bareHost) {
+    this.bareHost = bareHost;
+  }
+
+  /**
+   * State of the container of an application.
+   **/
+  public Container state(ContainerState state) {
+    this.state = state;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "State of the container of an application.")
+  @JsonProperty("state")
+  public ContainerState getState() {
+    return state;
+  }
+
+  public void setState(ContainerState state) {
+    this.state = state;
+  }
+
+  /**
+   * Name of the component that this container instance belongs to.
+   **/
+  public Container componentName(String componentName) {
+    this.componentName = componentName;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Name of the component that this container instance belongs to.")
+  @JsonProperty("component_name")
+  public String getComponentName() {
+    return componentName;
+  }
+
+  @XmlElement(name = "component_name")
+  public void setComponentName(String componentName) {
+    this.componentName = componentName;
+  }
+
+  /**
+   * Resource used for this container.
+   **/
+  public Container resource(Resource resource) {
+    this.resource = resource;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Resource used for this container.")
+  @JsonProperty("resource")
+  public Resource getResource() {
+    return resource;
+  }
+
+  public void setResource(Resource resource) {
+    this.resource = resource;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Container container = (Container) o;
+    return Objects.equals(this.id, container.id)
+        && Objects.equals(this.launchTime, container.launchTime)
+        && Objects.equals(this.ip, container.ip)
+        && Objects.equals(this.hostname, container.hostname)
+        && Objects.equals(this.bareHost, container.bareHost)
+        && Objects.equals(this.state, container.state)
+        && Objects.equals(this.componentName, container.componentName)
+        && Objects.equals(this.resource, container.resource);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(id, launchTime, ip, hostname, bareHost, state,
+        componentName, resource);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Container {\n");
+
+    sb.append("    id: ").append(toIndentedString(id)).append("\n");
+    sb.append("    launchTime: ").append(toIndentedString(launchTime))
+        .append("\n");
+    sb.append("    ip: ").append(toIndentedString(ip)).append("\n");
+    sb.append("    hostname: ").append(toIndentedString(hostname)).append("\n");
+    sb.append("    bareHost: ").append(toIndentedString(bareHost)).append("\n");
+    sb.append("    state: ").append(toIndentedString(state)).append("\n");
+    sb.append("    componentName: ").append(toIndentedString(componentName))
+        .append("\n");
+    sb.append("    resource: ").append(toIndentedString(resource)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ContainerState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ContainerState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ContainerState.java
new file mode 100644
index 0000000..cb017fb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ContainerState.java
@@ -0,0 +1,25 @@
+/*
+ * 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.services.resource;
+
+/**
+ * The current state of the container of an application.
+ **/
+public enum ContainerState {
+  INIT, READY;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Error.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Error.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Error.java
new file mode 100644
index 0000000..91c4e3a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/Error.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.services.resource;
+
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+public class Error {
+
+  private Integer code = null;
+  private String message = null;
+  private String fields = null;
+
+  /**
+   **/
+  public Error code(Integer code) {
+    this.code = code;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "")
+  @JsonProperty("code")
+  public Integer getCode() {
+    return code;
+  }
+
+  public void setCode(Integer code) {
+    this.code = code;
+  }
+
+  /**
+   **/
+  public Error message(String message) {
+    this.message = message;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "")
+  @JsonProperty("message")
+  public String getMessage() {
+    return message;
+  }
+
+  public void setMessage(String message) {
+    this.message = message;
+  }
+
+  /**
+   **/
+  public Error fields(String fields) {
+    this.fields = fields;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "")
+  @JsonProperty("fields")
+  public String getFields() {
+    return fields;
+  }
+
+  public void setFields(String fields) {
+    this.fields = fields;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Error error = (Error) o;
+    return Objects.equals(this.code, error.code)
+        && Objects.equals(this.message, error.message)
+        && Objects.equals(this.fields, error.fields);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(code, message, fields);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Error {\n");
+
+    sb.append("    code: ").append(toIndentedString(code)).append("\n");
+    sb.append("    message: ").append(toIndentedString(message)).append("\n");
+    sb.append("    fields: ").append(toIndentedString(fields)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/PlacementPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/PlacementPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/PlacementPolicy.java
new file mode 100644
index 0000000..7541e2f
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/PlacementPolicy.java
@@ -0,0 +1,97 @@
+/*
+ * 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.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Placement policy of an instance of an application. This feature is in the
+ * works in YARN-4902.
+ **/
+
+@ApiModel(description = "Placement policy of an instance of an application. This feature is in the works in YARN-4902.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+public class PlacementPolicy {
+
+  private String label = null;
+
+  /**
+   * Assigns an app to a named partition of the cluster where the application
+   * desires to run (optional). If not specified all apps are submitted to a
+   * default label of the app owner. One or more labels can be setup for each
+   * application owner account with required constraints like no-preemption,
+   * sla-99999, preemption-ok, etc.
+   **/
+  public PlacementPolicy label(String label) {
+    this.label = label;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Assigns an app to a named partition of the cluster where the application desires to run (optional). If not specified all apps are submitted to a default label of the app owner. One or more labels can be setup for each application owner account with required constraints like no-preemption, sla-99999, preemption-ok, etc.")
+  @JsonProperty("label")
+  public String getLabel() {
+    return label;
+  }
+
+  public void setLabel(String label) {
+    this.label = label;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    PlacementPolicy placementPolicy = (PlacementPolicy) o;
+    return Objects.equals(this.label, placementPolicy.label);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(label);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class PlacementPolicy {\n");
+
+    sb.append("    label: ").append(toIndentedString(label)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
new file mode 100644
index 0000000..80fdf92
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/resource/ReadinessCheck.java
@@ -0,0 +1,161 @@
+/*
+ * 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.services.resource;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonValue;
+
+/**
+ * A custom command or a pluggable helper container to determine the readiness
+ * of a container of a component. Readiness for every application is different.
+ * Hence the need for a simple interface, with scope to support advanced
+ * usecases.
+ **/
+
+@ApiModel(description = "A custom command or a pluggable helper container to determine the readiness of a container of a component. Readiness for every application is different. Hence the need for a simple interface, with scope to support advanced usecases.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+public class ReadinessCheck {
+
+  public enum TypeEnum {
+    HTTP("http");
+
+    private String value;
+
+    TypeEnum(String value) {
+      this.value = value;
+    }
+
+    @Override
+    @JsonValue
+    public String toString() {
+      return value;
+    }
+  }
+
+  private TypeEnum type = null;
+  private String uri = null;
+  private Artifact artifact = null;
+
+  /**
+   * http (YARN will perform a simple REST call at a regular interval and expect
+   * a 204 No content).
+   **/
+  public ReadinessCheck type(TypeEnum type) {
+    this.type = type;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "http (YARN will perform a simple REST call at a regular interval and expect a 204 No content).")
+  @JsonProperty("type")
+  public TypeEnum getType() {
+    return type;
+  }
+
+  public void setType(TypeEnum type) {
+    this.type = type;
+  }
+
+  /**
+   * Fully qualified REST uri endpoint.
+   **/
+  public ReadinessCheck uri(String uri) {
+    this.uri = uri;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", required = true, value = "Fully qualified REST uri endpoint.")
+  @JsonProperty("uri")
+  public String getUri() {
+    return uri;
+  }
+
+  public void setUri(String uri) {
+    this.uri = uri;
+  }
+
+  /**
+   * Artifact of the pluggable readiness check helper container (optional). If
+   * specified, this helper container typically hosts the http uri and
+   * encapsulates the complex scripts required to perform actual container
+   * readiness check. At the end it is expected to respond a 204 No content just
+   * like the simplified use case. This pluggable framework benefits application
+   * owners who can run applications without any packaging modifications. Note,
+   * artifacts of type docker only is supported for now.
+   **/
+  public ReadinessCheck artifact(Artifact artifact) {
+    this.artifact = artifact;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Artifact of the pluggable readiness check helper container (optional). If specified, this helper container typically hosts the http uri and encapsulates the complex scripts required to perform actual container readiness check. At the end it is expected to respond a 204 No content just like the simplified use case. This pluggable framework benefits application owners who can run applications without any packaging modifications. Note, artifacts of type docker only is supported for now.")
+  @JsonProperty("artifact")
+  public Artifact getArtifact() {
+    return artifact;
+  }
+
+  public void setArtifact(Artifact artifact) {
+    this.artifact = artifact;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    ReadinessCheck readinessCheck = (ReadinessCheck) o;
+    return Objects.equals(this.type, readinessCheck.type)
+        && Objects.equals(this.uri, readinessCheck.uri)
+        && Objects.equals(this.artifact, readinessCheck.artifact);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(type, uri, artifact);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class ReadinessCheck {\n");
+
+    sb.append("    type: ").append(toIndentedString(type)).append("\n");
+    sb.append("    uri: ").append(toIndentedString(uri)).append("\n");
+    sb.append("    artifact: ").append(toIndentedString(artifact)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[3/3] hadoop git commit: YARN-5610. Initial code for native services REST API. Contributed by Gour Saha

Posted by ji...@apache.org.
YARN-5610. Initial code for native services REST API. Contributed by Gour Saha


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

Branch: refs/heads/yarn-native-services
Commit: 4d5ded9f819510677a9f1a40f72f1e88a5c11a7c
Parents: 625b92c
Author: Jian He <ji...@apache.org>
Authored: Tue Oct 11 11:36:57 2016 -0700
Committer: Jian He <ji...@apache.org>
Committed: Tue Oct 11 11:36:57 2016 -0700

----------------------------------------------------------------------
 hadoop-project/pom.xml                          |   20 +
 .../dev-support/findbugs-exclude.xml            |   20 +
 .../hadoop-yarn-services-api/pom.xml            |  225 +++
 .../yarn/services/api/ApplicationApi.java       |   38 +
 .../api/impl/ApplicationApiService.java         | 1527 ++++++++++++++++++
 .../yarn/services/resource/Application.java     |  452 ++++++
 .../services/resource/ApplicationState.java     |   25 +
 .../services/resource/ApplicationStatus.java    |  147 ++
 .../hadoop/yarn/services/resource/Artifact.java |  155 ++
 .../yarn/services/resource/BaseResource.java    |   48 +
 .../yarn/services/resource/Component.java       |  377 +++++
 .../yarn/services/resource/ConfigFile.java      |  190 +++
 .../yarn/services/resource/Configuration.java   |  147 ++
 .../yarn/services/resource/Container.java       |  256 +++
 .../yarn/services/resource/ContainerState.java  |   25 +
 .../hadoop/yarn/services/resource/Error.java    |  125 ++
 .../yarn/services/resource/PlacementPolicy.java |   97 ++
 .../yarn/services/resource/ReadinessCheck.java  |  161 ++
 .../hadoop/yarn/services/resource/Resource.java |  149 ++
 .../yarn/services/utils/RestApiConstants.java   |   66 +
 .../services/utils/RestApiErrorMessages.java    |   79 +
 .../services/webapp/ApplicationApiWebApp.java   |  127 ++
 .../src/main/resources/log4j-server.properties  |   76 +
 .../resources/webapps/services-rest-api/app     |   16 +
 .../src/main/scripts/run_rest_service.sh        |   28 +
 .../src/main/webapp/WEB-INF/web.xml             |   36 +
 .../api/impl/TestApplicationApiService.java     |  232 +++
 .../hadoop-yarn-applications/pom.xml            |    2 +-
 28 files changed, 4845 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-project/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml
index dee79f7..4e9dfaa 100644
--- a/hadoop-project/pom.xml
+++ b/hadoop-project/pom.xml
@@ -122,6 +122,9 @@
     <native-maven-plugin.version>1.0-alpha-8</native-maven-plugin.version>
     <surefire.fork.timeout>900</surefire.fork.timeout>
     <aws-java-sdk.version>1.10.6</aws-java-sdk.version>
+
+    <swagger-annotations-version>1.5.4</swagger-annotations-version>
+    <maven-doxia-module-markdown.version>1.4</maven-doxia-module-markdown.version>
   </properties>
 
   <dependencyManagement>
@@ -1151,6 +1154,23 @@
           <artifactId>kerb-simplekdc</artifactId>
           <version>1.0.0-RC2</version>
         </dependency>
+
+      <dependency>
+        <groupId>io.swagger</groupId>
+        <artifactId>swagger-annotations</artifactId>
+        <version>${swagger-annotations-version}</version>
+      </dependency>
+      <dependency>
+        <groupId>com.fasterxml.jackson.jaxrs</groupId>
+        <artifactId>jackson-jaxrs-json-provider</artifactId>
+        <version>${jackson2.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.maven.doxia</groupId>
+        <artifactId>doxia-module-markdown</artifactId>
+        <version>${maven-doxia-module-markdown.version}</version>
+      </dependency>
+
     </dependencies>
   </dependencyManagement>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/dev-support/findbugs-exclude.xml
new file mode 100644
index 0000000..b89146a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/dev-support/findbugs-exclude.xml
@@ -0,0 +1,20 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+   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.
+-->
+<FindBugsFilter>
+
+</FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml
new file mode 100644
index 0000000..78b7855
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/pom.xml
@@ -0,0 +1,225 @@
+<!--
+   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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.hadoop</groupId>
+    <artifactId>hadoop-yarn-applications</artifactId>
+    <version>3.0.0-alpha2-SNAPSHOT</version>
+  </parent>
+  <groupId>org.apache.hadoop</groupId>
+  <artifactId>hadoop-yarn-services-api</artifactId>
+  <name>Apache Hadoop YARN Services API</name>
+  <version>3.0.0-alpha2-SNAPSHOT</version>
+  <packaging>jar</packaging>
+  <description>Hadoop YARN REST APIs for services</description>
+
+  <properties>
+    <test.failIfNoTests>false</test.failIfNoTests>
+    <powermock.version>1.6.5</powermock.version>
+  </properties>
+
+  <build>
+
+    <!-- resources are filtered for dynamic updates. This gets build info in-->
+    <resources>
+      <resource>
+        <directory>src/main/resources</directory>
+        <filtering>true</filtering>
+      </resource>
+      <resource>
+        <directory>src/main/scripts/</directory>
+        <filtering>true</filtering>
+      </resource>
+    </resources>
+
+    <plugins>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-jar-plugin</artifactId>
+        <version>${maven-jar-plugin.version}</version>
+        <!-- The configuration of the plugin -->
+        <configuration>
+          <!-- Configuration of the archiver -->
+          <archive>
+            <manifestEntries>
+              <mode>development</mode>
+              <url>${project.url}</url>
+            </manifestEntries>
+            <!-- Manifest specific configuration -->
+            <manifest>
+            </manifest>
+          </archive>
+          <excludes>
+            <exclude>**/run_rest_service.sh</exclude>
+          </excludes>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>test-jar</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>${maven-surefire-plugin.version}</version>
+        <configuration>
+          <reuseForks>${test.reuseForks}</reuseForks>
+          <forkMode>${test.forkMode}</forkMode>
+          <forkCount>1</forkCount>
+          <forkedProcessTimeoutInSeconds>${test.forkedProcessTimeoutInSeconds}
+          </forkedProcessTimeoutInSeconds>
+          <threadCount>1</threadCount>
+          <argLine>${test.argLine}</argLine>
+          <failIfNoTests>${test.failIfNoTests}</failIfNoTests>
+          <redirectTestOutputToFile>${build.redirect.test.output.to.file}</redirectTestOutputToFile>
+          <environmentVariables>
+            <PATH>${test.env.path}</PATH>
+          </environmentVariables>
+          <systemPropertyVariables>
+            <java.net.preferIPv4Stack>true</java.net.preferIPv4Stack>
+            <java.awt.headless>true</java.awt.headless>
+          </systemPropertyVariables>
+          <includes>
+            <include>**/Test*.java</include>
+          </includes>
+          <excludes>
+            <exclude>**/Test*$*.java</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <version>${maven-site-plugin.version}</version>
+
+        <dependencies>
+          <dependency>
+            <groupId>org.apache.maven.doxia</groupId>
+            <artifactId>doxia-module-markdown</artifactId>
+            <version>${maven-doxia-module-markdown.version}</version>
+          </dependency>
+        </dependencies>
+      </plugin>
+    </plugins>
+  </build>
+
+  <reporting>
+  </reporting>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-yarn-slider-core</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <type>test-jar</type>
+      <scope>test</scope>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.swagger</groupId>
+      <artifactId>swagger-annotations</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.jaxrs</groupId>
+      <artifactId>jackson-jaxrs-json-provider</artifactId>
+      <exclusions>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.jaxrs</groupId>
+          <artifactId>jackson-jaxrs-base</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.powermock</groupId>
+      <artifactId>powermock-module-junit4</artifactId>
+      <version>${powermock.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.powermock</groupId>
+      <artifactId>powermock-api-easymock</artifactId>
+      <version>${powermock.version}</version>
+      <scope>test</scope>
+      <exclusions>
+        <exclusion>
+          <groupId>org.easymock</groupId>
+          <artifactId>easymock</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+  </dependencies>
+
+  <profiles>
+
+    <profile>
+      <id>rat</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.rat</groupId>
+            <artifactId>apache-rat-plugin</artifactId>
+            <version>${apache-rat-plugin.version}</version>
+            <executions>
+              <execution>
+                <id>check-licenses</id>
+                <goals>
+                  <goal>check</goal>
+                </goals>
+              </execution>
+            </executions>
+            <configuration>
+              <excludes>
+                <exclude>**/*.json</exclude>
+                <exclude>**/THIRD-PARTY.properties</exclude>
+              </excludes>
+            </configuration>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
+
+  </profiles>
+
+</project>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/ApplicationApi.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/ApplicationApi.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/ApplicationApi.java
new file mode 100644
index 0000000..654413c
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/ApplicationApi.java
@@ -0,0 +1,38 @@
+/*
+ * 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.services.api;
+
+import javax.ws.rs.core.Response;
+
+import org.apache.hadoop.yarn.services.resource.Application;
+
+/**
+ * Apache Hadoop YARN Services REST API interface.
+ *
+ */
+public interface ApplicationApi {
+  Response createApplication(Application application);
+
+  Response getApplications(String state);
+
+  Response getApplication(String appName);
+
+  Response deleteApplication(String appName);
+
+  Response updateApplication(String appName, Application updateAppData);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4d5ded9f/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
new file mode 100644
index 0000000..9645696
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/services/api/impl/ApplicationApiService.java
@@ -0,0 +1,1527 @@
+/*
+ * 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.services.api.impl;
+
+import static org.apache.hadoop.yarn.services.utils.RestApiConstants.*;
+import static org.apache.hadoop.yarn.services.utils.RestApiErrorMessages.*;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.regex.Pattern;
+
+import javax.inject.Singleton;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+
+import org.apache.commons.collections.MapUtils;
+import org.apache.commons.lang.SerializationUtils;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.services.api.ApplicationApi;
+import org.apache.hadoop.yarn.services.resource.Application;
+import org.apache.hadoop.yarn.services.resource.ApplicationState;
+import org.apache.hadoop.yarn.services.resource.ApplicationStatus;
+import org.apache.hadoop.yarn.services.resource.Artifact;
+import org.apache.hadoop.yarn.services.resource.Component;
+import org.apache.hadoop.yarn.services.resource.ConfigFile;
+import org.apache.hadoop.yarn.services.resource.Configuration;
+import org.apache.hadoop.yarn.services.resource.Container;
+import org.apache.hadoop.yarn.services.resource.ContainerState;
+import org.apache.hadoop.yarn.services.resource.Resource;
+import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.api.StateValues;
+import org.apache.slider.client.SliderClient;
+import org.apache.slider.common.SliderExitCodes;
+import org.apache.slider.common.params.ActionCreateArgs;
+import org.apache.slider.common.params.ActionFlexArgs;
+import org.apache.slider.common.params.ActionFreezeArgs;
+import org.apache.slider.common.params.ActionListArgs;
+import org.apache.slider.common.params.ActionRegistryArgs;
+import org.apache.slider.common.params.ActionStatusArgs;
+import org.apache.slider.common.params.ActionThawArgs;
+import org.apache.slider.common.params.ComponentArgsDelegate;
+import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.common.tools.SliderVersionInfo;
+import org.apache.slider.core.buildutils.BuildHelper;
+import org.apache.slider.core.exceptions.BadClusterStateException;
+import org.apache.slider.core.exceptions.NotFoundException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.exceptions.UnknownApplicationInstanceException;
+import org.apache.slider.core.registry.docstore.ConfigFormat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.GsonBuilder;
+import com.google.gson.JsonArray;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonNull;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParser;
+
+@Singleton
+@Path(APPLICATIONS_API_RESOURCE_PATH)
+@Consumes({ MediaType.APPLICATION_JSON })
+@Produces({ MediaType.APPLICATION_JSON })
+public class ApplicationApiService implements ApplicationApi {
+  private static final Logger logger = LoggerFactory
+      .getLogger(ApplicationApiService.class);
+  private static org.apache.hadoop.conf.Configuration SLIDER_CONFIG;
+  private static UserGroupInformation SLIDER_USER;
+  private static SliderClient SLIDER_CLIENT;
+
+  static {
+    init();
+  }
+
+  // initialize all the common resources - order is important
+  protected static void init() {
+    SLIDER_CONFIG = getSliderClientConfiguration();
+    SLIDER_USER = getSliderUser();
+    SLIDER_CLIENT = createSliderClient();
+  }
+
+  @GET
+  @Path("/slider-version")
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON })
+  public Response getSliderVersion() {
+    logger.info("GET: getSliderVersion");
+
+    Map<String, Object> metadata = new HashMap<>();
+    BuildHelper.addBuildMetadata(metadata, "org.apache.hadoop.yarn.services");
+    String sliderVersion = metadata.toString();
+    logger.info("Slider version = {}", sliderVersion);
+    String hadoopVersion = SliderVersionInfo.getHadoopVersionString();
+    logger.info("Hadoop version = {}", hadoopVersion);
+    return Response.ok(
+        "{ \"slider_version\": \"" + sliderVersion
+            + "\", \"hadoop_version\": \"" + hadoopVersion + "\"}").build();
+  }
+
+  @POST
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON })
+  public Response createApplication(Application application) {
+    logger.info("POST: createApplication for app = {}", application);
+    ApplicationStatus applicationStatus = new ApplicationStatus();
+
+    Map<String, String> compNameArtifactIdMap = new HashMap<>();
+    // post payload validation
+    try {
+      validateApplicationPostPayload(application, compNameArtifactIdMap);
+    } catch (IllegalArgumentException e) {
+      applicationStatus.setDiagnostics(e.getMessage());
+      return Response.status(Status.BAD_REQUEST).entity(applicationStatus)
+          .build();
+    }
+    String applicationId = null;
+    try {
+      applicationId = createSliderApp(application, compNameArtifactIdMap);
+      applicationStatus.setState(ApplicationState.ACCEPTED);
+    } catch (SliderException se) {
+      logger.error("Create application failed", se);
+      if (se.getExitCode() == SliderExitCodes.EXIT_APPLICATION_IN_USE) {
+        applicationStatus.setDiagnostics(ERROR_APPLICATION_IN_USE);
+        return Response.status(Status.BAD_REQUEST).entity(applicationStatus)
+            .build();
+      } else {
+        applicationStatus.setDiagnostics(se.getMessage());
+      }
+    } catch (Exception e) {
+      logger.error("Create application failed", e);
+      applicationStatus.setDiagnostics(e.getMessage());
+    }
+
+    if (StringUtils.isNotEmpty(applicationId)) {
+      applicationStatus.setUri(CONTEXT_ROOT + APPLICATIONS_API_RESOURCE_PATH
+          + "/" + application.getName());
+      // 202 = ACCEPTED
+      return Response.status(HTTP_STATUS_CODE_ACCEPTED)
+          .entity(applicationStatus).build();
+    } else {
+      return Response.status(Status.INTERNAL_SERVER_ERROR)
+          .entity(applicationStatus).build();
+    }
+  }
+
+  @VisibleForTesting
+  protected void validateApplicationPostPayload(Application application,
+      Map<String, String> compNameArtifactIdMap) {
+    if (StringUtils.isEmpty(application.getName())) {
+      throw new IllegalArgumentException(ERROR_APPLICATION_NAME_INVALID);
+    }
+    if (!SliderUtils.isClusternameValid(application.getName())) {
+      throw new IllegalArgumentException(ERROR_APPLICATION_NAME_INVALID_FORMAT);
+    }
+
+    // If the application has no components do top-level checks
+    if (application.getComponents() == null) {
+      // artifact
+      if (application.getArtifact() == null) {
+        throw new IllegalArgumentException(ERROR_ARTIFACT_INVALID);
+      }
+      if (StringUtils.isEmpty(application.getArtifact().getId())) {
+        throw new IllegalArgumentException(ERROR_ARTIFACT_ID_INVALID);
+      }
+
+      // If artifact is of type APPLICATION, add a slider specific property
+      if (application.getArtifact().getType() == Artifact.TypeEnum.APPLICATION) {
+        if (application.getConfiguration() == null) {
+          application.setConfiguration(new Configuration());
+        }
+        addPropertyToConfiguration(application.getConfiguration(),
+            PROPERTY_COMPONENT_TYPE, COMPONENT_TYPE_EXTERNAL);
+      }
+      // resource
+      validateApplicationResource(application.getResource(), null, application
+          .getArtifact().getType());
+
+      // container size
+      if (application.getNumberOfContainers() == null) {
+        throw new IllegalArgumentException(ERROR_CONTAINERS_COUNT_INVALID);
+      }
+    } else {
+      // If the application has components, then run checks for each component.
+      // Let global values take effect if component level values are not
+      // provided.
+      Artifact globalArtifact = application.getArtifact();
+      Resource globalResource = application.getResource();
+      Long globalNumberOfContainers = application.getNumberOfContainers();
+      for (Component comp : application.getComponents()) {
+        // artifact
+        if (comp.getArtifact() == null) {
+          comp.setArtifact(globalArtifact);
+        }
+        // If still null raise validation exception
+        if (comp.getArtifact() == null) {
+          throw new IllegalArgumentException(String.format(
+              ERROR_ARTIFACT_FOR_COMP_INVALID, comp.getName()));
+        }
+        if (StringUtils.isEmpty(comp.getArtifact().getId())) {
+          throw new IllegalArgumentException(String.format(
+              ERROR_ARTIFACT_ID_FOR_COMP_INVALID, comp.getName()));
+        }
+
+        // If artifact is of type APPLICATION, add a slider specific property
+        if (comp.getArtifact().getType() == Artifact.TypeEnum.APPLICATION) {
+          if (comp.getConfiguration() == null) {
+            comp.setConfiguration(new Configuration());
+          }
+          addPropertyToConfiguration(comp.getConfiguration(),
+              PROPERTY_COMPONENT_TYPE, COMPONENT_TYPE_EXTERNAL);
+          compNameArtifactIdMap.put(comp.getName(), comp.getArtifact().getId());
+          comp.setName(comp.getArtifact().getId());
+        }
+
+        // resource
+        if (comp.getResource() == null) {
+          comp.setResource(globalResource);
+        }
+        validateApplicationResource(comp.getResource(), comp, comp
+            .getArtifact().getType());
+
+        // container count
+        if (comp.getNumberOfContainers() == null) {
+          comp.setNumberOfContainers(globalNumberOfContainers);
+        }
+        if (comp.getNumberOfContainers() == null) {
+          throw new IllegalArgumentException(String.format(
+              ERROR_CONTAINERS_COUNT_FOR_COMP_INVALID, comp.getName()));
+        }
+      }
+    }
+
+    // If it is a simple app with no components, then create a default component
+    if (application.getComponents() == null) {
+      application.setComponents(getDefaultComponentAsList());
+    }
+
+    // Application lifetime if not specified, is set to unlimited lifetime
+    if (application.getLifetime() == null) {
+      application.setLifetime(DEFAULT_UNLIMITED_LIFETIME);
+    }
+  }
+
+  private void validateApplicationResource(Resource resource, Component comp,
+      Artifact.TypeEnum artifactType) {
+    // Only apps/components of type APPLICATION can skip resource requirement
+    if (resource == null && artifactType == Artifact.TypeEnum.APPLICATION) {
+      return;
+    }
+    if (resource == null) {
+      throw new IllegalArgumentException(comp == null ? ERROR_RESOURCE_INVALID
+          : String.format(ERROR_RESOURCE_FOR_COMP_INVALID, comp.getName()));
+    }
+    // One and only one of profile OR cpus & memory can be specified. Specifying
+    // both raises validation error.
+    if (StringUtils.isNotEmpty(resource.getProfile())
+        && (resource.getCpus() != null
+            || StringUtils.isNotEmpty(resource.getMemory()))) {
+      throw new IllegalArgumentException(
+          comp == null ? ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_NOT_SUPPORTED
+              : String.format(
+                  ERROR_RESOURCE_PROFILE_MULTIPLE_VALUES_FOR_COMP_NOT_SUPPORTED,
+                  comp.getName()));
+    }
+    // Currently resource profile is not supported yet, so we will raise
+    // validation error if only resource profile is specified
+    if (StringUtils.isNotEmpty(resource.getProfile())) {
+      throw new IllegalArgumentException(
+          ERROR_RESOURCE_PROFILE_NOT_SUPPORTED_YET);
+    }
+
+    String memory = resource.getMemory();
+    Integer cpus = resource.getCpus();
+    if (StringUtils.isEmpty(memory)) {
+      throw new IllegalArgumentException(
+          comp == null ? ERROR_RESOURCE_MEMORY_INVALID : String.format(
+              ERROR_RESOURCE_MEMORY_FOR_COMP_INVALID, comp.getName()));
+    }
+    if (cpus == null) {
+      throw new IllegalArgumentException(
+          comp == null ? ERROR_RESOURCE_CPUS_INVALID : String.format(
+              ERROR_RESOURCE_CPUS_FOR_COMP_INVALID, comp.getName()));
+    }
+    if (cpus <= 0) {
+      throw new IllegalArgumentException(
+          comp == null ? ERROR_RESOURCE_CPUS_INVALID_RANGE : String.format(
+              ERROR_RESOURCE_CPUS_FOR_COMP_INVALID_RANGE, comp.getName()));
+    }
+  }
+
+  private String createSliderApp(Application application,
+      Map<String, String> compNameArtifactIdMap) throws IOException,
+      YarnException, InterruptedException {
+    final String appName = application.getName();
+    final String queueName = application.getQueue();
+
+    final ActionCreateArgs createArgs = new ActionCreateArgs();
+    addAppConfOptions(createArgs, application, compNameArtifactIdMap);
+    addResourceOptions(createArgs, application);
+    String metainfoJson = getMetainfoJson(application, compNameArtifactIdMap);
+
+    createArgs.appMetaInfoJson = metainfoJson;
+    if (queueName != null && queueName.trim().length() > 0) {
+      createArgs.queue = queueName.trim();
+    }
+
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<String>() {
+      @Override
+      public String run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        sliderClient.actionCreate(appName, createArgs);
+        ApplicationId applicationId = sliderClient.applicationId;
+        if (applicationId != null) {
+          return applicationId.toString();
+          // return getApplicationIdString(applicationId);
+        }
+        return null;
+      }
+    });
+  }
+
+  private void addAppConfOptions(ActionCreateArgs createArgs,
+      Application application, Map<String, String> compNameArtifactIdMap) throws IOException {
+    List<String> appCompOptionTriples = createArgs.optionsDelegate.compOptTriples; // TODO: optionTuples instead of compOptTriples
+    logger.info("Initial appCompOptionTriples = {}",
+        Arrays.toString(appCompOptionTriples.toArray()));
+    List<String> appOptions = createArgs.optionsDelegate.optionTuples;
+    logger.info("Initial appOptions = {}",
+        Arrays.toString(appOptions.toArray()));
+    // TODO: Set Slider-AM memory and vcores here
+    //    appCompOptionTriples.addAll(Arrays.asList(SLIDER_APPMASTER_COMPONENT_NAME,
+    //        "", ""));
+
+    // Global configuration - for override purpose
+    // TODO: add it to yaml
+    Configuration globalConfig = null;
+    //    Configuration globalConfig = (Configuration) SerializationUtils
+    //        .clone(application.getConfiguration());
+
+    // TODO: Add the below into globalConfig
+    //    if (application.getConfigurations() != null) {
+    //      for (Entry<String, String> entry : application.getConfigurations()
+    //          .entrySet()) {
+    //        globalConf.addProperty(entry.getKey(), entry.getValue());
+    //      }
+    //    }
+
+    Set<String> uniqueGlobalPropertyCache = new HashSet<>();
+    if (application.getConfiguration() != null
+        && application.getConfiguration().getProperties() != null) {
+      for (Map.Entry<String, String> propEntry : application.getConfiguration()
+          .getProperties().entrySet()) {
+        addOptionsIfNotPresent(appOptions, uniqueGlobalPropertyCache,
+            propEntry.getKey(), propEntry.getValue());
+      }
+    }
+    if (application.getComponents() != null) {
+      Map<String, String> placeholders = new HashMap<>();
+      for (Component comp : application.getComponents()) {
+        if (Boolean.TRUE.equals(comp.getUniqueComponentSupport())) {
+          for (int i = 1; i <= comp.getNumberOfContainers(); i++) {
+            placeholders.put(PLACEHOLDER_COMPONENT_ID, Integer.toString(i));
+            appCompOptionTriples.addAll(createAppConfigComponent(
+                comp.getName() + i, comp, comp.getName() + i, globalConfig,
+                placeholders, compNameArtifactIdMap));
+          }
+        } else {
+          appCompOptionTriples.addAll(createAppConfigComponent(comp.getName(),
+              comp, comp.getName(), globalConfig, null, compNameArtifactIdMap));
+        }
+      }
+    }
+
+    logger.info("Updated appCompOptionTriples = {}",
+        Arrays.toString(appCompOptionTriples.toArray()));
+    logger.info("Updated appOptions = {}",
+        Arrays.toString(appOptions.toArray()));
+  }
+
+  private void addOptionsIfNotPresent(List<String> options,
+      Set<String> uniqueGlobalPropertyCache, String key, String value) {
+    if (uniqueGlobalPropertyCache == null) {
+      options.addAll(Arrays.asList(key, value));
+    } else if (!uniqueGlobalPropertyCache.contains(key)) {
+      options.addAll(Arrays.asList(key, value));
+      uniqueGlobalPropertyCache.add(key);
+    }
+  }
+
+  private void addPropertyToConfiguration(Configuration conf, String key,
+      String value) {
+    if (conf == null) {
+      return;
+    }
+    if (conf.getProperties() == null) {
+      conf.setProperties(new HashMap<String, String>());
+    }
+    conf.getProperties().put(key, value);
+  }
+
+  private List<String> createAppConfigComponent(String compName,
+      Component component, String configPrefix, Configuration globalConf,
+      Map<String, String> placeholders,
+      Map<String, String> compNameArtifactIdMap) {
+    List<String> appConfOptTriples = new ArrayList<>();
+
+    if (component.getConfiguration() != null
+        && component.getConfiguration().getProperties() != null) {
+      for (Map.Entry<String, String> propEntry : component.getConfiguration()
+          .getProperties().entrySet()) {
+        appConfOptTriples.addAll(Arrays.asList(compName, propEntry.getKey(),
+            replacePlaceholders(propEntry.getValue(), placeholders)));
+      }
+    }
+
+    // If artifact is of type APPLICATION, then in the POST JSON there will
+    // be no component definition for that artifact. Hence it's corresponding id
+    // field is added. Every external APPLICATION has a unique id field.
+    List<String> convertedDeps = new ArrayList<>();
+    for (String dep : component.getDependencies()) {
+      if (compNameArtifactIdMap.containsKey(dep)) {
+        convertedDeps.add(compNameArtifactIdMap.get(dep));
+      } else {
+        convertedDeps.add(dep);
+      }
+    }
+    // If the DNS dependency property is set to true for a component, it means
+    // that it is ensured that DNS entry has been added for all the containers
+    // of this component, before moving on to the next component in the DAG.
+    if (hasPropertyWithValue(component, PROPERTY_DNS_DEPENDENCY, "true")) {
+      if (component.getArtifact().getType() == Artifact.TypeEnum.APPLICATION) {
+        convertedDeps.add(component.getArtifact().getId());
+      } else {
+        convertedDeps.add(compName);
+      }
+    }
+    if (convertedDeps.size() > 0) {
+      appConfOptTriples.addAll(Arrays.asList(compName, "requires",
+          StringUtils.join(convertedDeps, ",")));
+    }
+    return appConfOptTriples;
+  }
+
+  private String replacePlaceholders(String value,
+      Map<String, String> placeholders) {
+    if (StringUtils.isEmpty(value) || placeholders == null) {
+      return value;
+    }
+    for (Map.Entry<String, String> placeholder : placeholders.entrySet()) {
+      value = value.replaceAll(Pattern.quote(placeholder.getKey()),
+          placeholder.getValue());
+    }
+    return value;
+  }
+
+  private List<String> createAppConfigGlobal(Component component,
+      Configuration globalConf, Set<String> uniqueGlobalPropertyCache) {
+    List<String> appOptions = new ArrayList<>();
+    if (component.getConfiguration() != null
+        && component.getConfiguration().getProperties() != null) {
+      for (Map.Entry<String, String> propEntry : component.getConfiguration()
+          .getProperties().entrySet()) {
+        addOptionsIfNotPresent(appOptions, uniqueGlobalPropertyCache,
+            propEntry.getKey(), propEntry.getValue());
+      }
+    }
+    return appOptions;
+  }
+
+  private void addResourceOptions(ActionCreateArgs createArgs,
+      Application application) throws IOException {
+    List<String> resCompOptionTriples = createArgs.optionsDelegate.resCompOptTriples;
+    logger.info("Initial resCompOptTriples = {}",
+        Arrays.toString(resCompOptionTriples.toArray()));
+    // TODO: Add any Slider AM resource specific props here like jvm.heapsize
+    //    resCompOptionTriples.addAll(Arrays.asList(SLIDER_APPMASTER_COMPONENT_NAME,
+    //        "", ""));
+
+    // Global resource - for override purpose
+    Resource globalResource = (Resource) SerializationUtils.clone(application
+        .getResource());
+    // Priority seeded with 1, expecting every new component will increase it by
+    // 1 making it ready for the next component to use.
+    if (application.getComponents() != null) {
+      int priority = 1;
+      for (Component comp : application.getComponents()) {
+        if (hasPropertyWithValue(comp, PROPERTY_COMPONENT_TYPE,
+            COMPONENT_TYPE_EXTERNAL)) {
+          continue;
+        }
+        if (Boolean.TRUE.equals(comp.getUniqueComponentSupport())) {
+          for (int i = 1; i <= comp.getNumberOfContainers(); i++) {
+            resCompOptionTriples.addAll(createResourcesComponent(comp.getName()
+                + i, comp, priority, 1, globalResource));
+            priority++;
+          }
+        } else {
+          resCompOptionTriples.addAll(createResourcesComponent(comp.getName(),
+              comp, priority, comp.getNumberOfContainers(), globalResource));
+          priority++;
+        }
+      }
+    }
+
+    logger.info("Updated resCompOptTriples = {}",
+        Arrays.toString(resCompOptionTriples.toArray()));
+  }
+
+  private boolean hasPropertyWithValue(Component comp, String key, String value) {
+    if (comp == null || key == null) {
+      return false;
+    }
+    if (comp.getConfiguration() == null
+        || comp.getConfiguration().getProperties() == null) {
+      return false;
+    }
+    Map<String, String> props = comp.getConfiguration().getProperties();
+    if (props.containsKey(key)) {
+      if (value == null) {
+        return props.get(key) == null;
+      } else {
+        if (value.equals(props.get(key))) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  private List<String> createResourcesComponent(String compName,
+      Component component, int priority, long numInstances,
+      Resource globalResource) {
+    String memory = component.getResource() == null ? globalResource
+        .getMemory() : component.getResource().getMemory();
+    Integer cpus = component.getResource() == null ? globalResource.getCpus()
+        : component.getResource().getCpus();
+
+    List<String> resCompOptTriples = new ArrayList<String>();
+    resCompOptTriples.addAll(Arrays.asList(compName,
+        ResourceKeys.COMPONENT_PRIORITY, Integer.toString(priority)));
+    resCompOptTriples.addAll(Arrays.asList(compName,
+        ResourceKeys.COMPONENT_INSTANCES, Long.toString(numInstances)));
+    resCompOptTriples.addAll(Arrays.asList(compName, ResourceKeys.YARN_MEMORY,
+        memory));
+    resCompOptTriples.addAll(Arrays.asList(compName, ResourceKeys.YARN_CORES,
+        cpus.toString()));
+    if (component.getPlacementPolicy() != null) {
+      resCompOptTriples.addAll(Arrays.asList(compName,
+          ResourceKeys.COMPONENT_PLACEMENT_POLICY,
+          component.getPlacementPolicy().getLabel()));
+    }
+
+    return resCompOptTriples;
+  }
+
+  private String getMetainfoJson(Application application,
+      Map<String, String> compNameArtifactIdMap) throws SliderException,
+      IOException {
+    JsonObject rootObj = new JsonObject();
+    rootObj.addProperty("schemaVersion", METAINFO_SCHEMA_VERSION);
+    JsonObject applicationObj = new JsonObject();
+    rootObj.add("application", applicationObj);
+    applicationObj.addProperty("name", application.getName().toUpperCase());
+    JsonArray componentsArray = new JsonArray();
+    applicationObj.add("components", componentsArray);
+    JsonArray commandOrdersArray = new JsonArray();
+    applicationObj.add("commandOrders", commandOrdersArray);
+
+    JsonArray exportGroupsArray = new JsonArray();
+    applicationObj.add("exportGroups", exportGroupsArray);
+    // Use only one export group
+    JsonObject exportGroup = new JsonObject();
+    exportGroup.addProperty("name", EXPORT_GROUP_NAME);
+    exportGroupsArray.add(exportGroup);
+    JsonArray exportsArray = new JsonArray();
+    exportGroup.add("exports", exportsArray);
+
+    if (application.getComponents() != null) {
+
+      // Set exports at application level
+      Map<String, String> appQuicklinks = application.getQuicklinks();
+      Map<String, String> placeholders = new HashMap<>();
+      placeholders.put(PLACEHOLDER_APP_NAME, application.getName());
+      if (appQuicklinks != null) {
+        for (Map.Entry<String, String> quicklink : appQuicklinks.entrySet()) {
+          JsonObject export = new JsonObject();
+          export.addProperty("name", quicklink.getKey());
+          export.addProperty("value",
+              replacePlaceholders(quicklink.getValue(), placeholders));
+          exportsArray.add(export);
+        }
+      }
+
+      for (Component comp : application.getComponents()) {
+        JsonObject compObject = null;
+        if (!hasPropertyWithValue(comp, PROPERTY_COMPONENT_TYPE,
+            COMPONENT_TYPE_EXTERNAL)) {
+          if (Boolean.TRUE.equals(comp.getUniqueComponentSupport())) {
+            for (int i = 1; i <= comp.getNumberOfContainers(); i++) {
+              // we also need the capability to specify ports and mount points
+              // sometime
+              compObject = createMetainfoComponent(comp, application,
+                  comp.getName() + i);
+              componentsArray.add(compObject);
+            }
+          } else {
+            compObject = createMetainfoComponent(comp, application,
+                comp.getName());
+            componentsArray.add(compObject);
+          }
+        }
+
+        // Translate dependencies into command orders
+        List<String> dependencies = comp.getDependencies();
+        if (dependencies != null && !dependencies.isEmpty()) {
+          JsonObject commandOrder = new JsonObject();
+          commandOrder.addProperty("command", comp.getName()
+              + COMMAND_ORDER_SUFFIX_START);
+          for (String dependency : dependencies) {
+            // If APPLICATION type artifact then map component name dependencies
+            // to artifact id
+            if (comp.getArtifact().getType() == Artifact.TypeEnum.APPLICATION) {
+              dependency = compNameArtifactIdMap.get(dependency);
+            }
+            commandOrder.addProperty("requires", dependency
+                + COMMAND_ORDER_SUFFIX_STARTED);
+          }
+          commandOrdersArray.add(commandOrder);
+        }
+
+        // Quicklinks need to be added as appExports and componentExports at the
+        // component level
+        List<String> compQuicklinks = comp.getQuicklinks();
+        if (compQuicklinks != null && !compQuicklinks.isEmpty()) {
+          if (MapUtils.isEmpty(appQuicklinks)) {
+            throw new SliderException(ERROR_QUICKLINKS_FOR_COMP_INVALID);
+          }
+          List<String> appExports = new ArrayList<>();
+          JsonArray compExportsArray = new JsonArray();
+          compObject.add("componentExports", compExportsArray);
+
+          for (String quicklink : compQuicklinks) {
+            appExports.add(EXPORT_GROUP_NAME + "-" + quicklink);
+
+            JsonObject compExport = new JsonObject();
+            compExport.addProperty("name", quicklink);
+            compExport.addProperty("value", appQuicklinks.get(quicklink));
+            compExportsArray.add(compExport);
+          }
+          compObject.addProperty("appExports",
+              StringUtils.join(appExports, ","));
+          // specify that there are published configs for this component
+          compObject.addProperty("publishConfig", "true");
+        }
+      }
+    }
+
+    String jsonString = new GsonBuilder().setPrettyPrinting().create()
+        .toJson(rootObj);
+    logger.info("Metainfo = \n{}", jsonString);
+    return jsonString;
+  }
+
+  private JsonObject createMetainfoComponent(Component comp,
+      Application application, String compName) {
+    JsonObject compObj = new JsonObject();
+    compObj.addProperty("name", compName);
+    // below is diff for each type
+    if (comp.getArtifact() != null && comp.getArtifact().getType() != null
+        && comp.getArtifact().getType() == Artifact.TypeEnum.DOCKER) {
+      compObj.addProperty("type", COMPONENT_TYPE_YARN_DOCKER);
+      JsonArray dockerContainerArray = new JsonArray();
+      compObj.add("dockerContainers", dockerContainerArray);
+      JsonObject dockerContainerObj = new JsonObject();
+      dockerContainerArray.add(dockerContainerObj);
+      dockerContainerObj.addProperty("name", compName.toLowerCase());
+      // if image not specified, then use global value
+      dockerContainerObj.addProperty("image",
+          comp.getArtifact().getId() == null ? application.getArtifact()
+              .getId() : comp.getArtifact().getId());
+      // If launch command not specified, then use global value. Resolve all
+      // placeholders.
+      Map<String, String> placeholders = new HashMap<>();
+      placeholders.put(PLACEHOLDER_APP_NAME, application.getName());
+      placeholders.put(PLACEHOLDER_APP_COMPONENT_NAME, compName);
+      dockerContainerObj.addProperty(
+          "startCommand",
+          comp.getLaunchCommand() == null ? replacePlaceholders(
+              application.getLaunchCommand(), placeholders)
+              : replacePlaceholders(comp.getLaunchCommand(), placeholders));
+      dockerContainerObj.addProperty("network", DEFAULT_NETWORK);
+      dockerContainerObj.addProperty("commandPath", DEFAULT_COMMAND_PATH);
+      // TODO: What to do with privContainer ?
+      dockerContainerObj.addProperty("runPrivilegedContainer",
+          comp.getRunPrivilegedContainer());
+      if (comp.getConfiguration() != null) {
+        List<ConfigFile> configFiles = comp.getConfiguration().getFiles();
+        if (configFiles != null && !configFiles.isEmpty()) {
+          JsonArray configFileArray = new JsonArray();
+          for (ConfigFile configFile : configFiles) {
+            JsonObject configFileObj = new JsonObject();
+            configFileObj.addProperty("type", configFile.getType().toString());
+            configFileObj.addProperty("fileName", configFile.getDestFile());
+            // TODO: add all properties which should include dictionaryName
+            configFileObj.addProperty("dictionaryName",
+                configFile.getSrcFile());
+            configFileArray.add(configFileObj);
+          }
+          dockerContainerObj.add("configFiles", configFileArray);
+        }
+      }
+      // we also need to specify artifact_management_service sometime
+    }
+    // we also need the capability to specify ports and mount points sometime
+    return compObj;
+  }
+
+  private static UserGroupInformation getSliderUser() {
+    if (SLIDER_USER != null) {
+      return SLIDER_USER;
+    }
+    UserGroupInformation sliderUser = null;
+    UserGroupInformation.setConfiguration(SLIDER_CONFIG);
+    String loggedInUser = getUserToRunAs();
+    try {
+      sliderUser = UserGroupInformation.getBestUGI(null, loggedInUser);
+      // TODO: Once plugged into RM process we should remove the previous call
+      // and replace it with getCurrentUser as commented below.
+      // sliderUser = UserGroupInformation.getCurrentUser();
+    } catch (IOException e) {
+      throw new RuntimeException("Unable to create UGI (slider user)", e);
+    }
+    return sliderUser;
+  }
+
+  private <T> T invokeSliderClientRunnable(
+      final SliderClientContextRunnable<T> runnable)
+      throws IOException, InterruptedException, YarnException {
+    try {
+      T value = SLIDER_USER.doAs(new PrivilegedExceptionAction<T>() {
+        @Override
+        public T run() throws Exception {
+          return runnable.run(SLIDER_CLIENT);
+        }
+      });
+      return value;
+    } catch (UndeclaredThrowableException e) {
+      Throwable cause = e.getCause();
+      if (cause instanceof YarnException) {
+        YarnException ye = (YarnException) cause;
+        throw ye;
+      }
+      throw e;
+    }
+  }
+
+  protected static SliderClient createSliderClient() {
+    if (SLIDER_CLIENT != null) {
+      return SLIDER_CLIENT;
+    }
+    org.apache.hadoop.conf.Configuration sliderClientConfiguration = SLIDER_CONFIG;
+    SliderClient client = new SliderClient() {
+      @Override
+      public void init(org.apache.hadoop.conf.Configuration conf) {
+        super.init(conf);
+        try {
+          initHadoopBinding();
+        } catch (SliderException e) {
+          throw new RuntimeException(
+              "Unable to automatically init Hadoop binding", e);
+        } catch (IOException e) {
+          throw new RuntimeException(
+              "Unable to automatically init Hadoop binding", e);
+        }
+      }
+    };
+    try {
+      logger
+          .debug("Slider Client configuration: {}", sliderClientConfiguration);
+      sliderClientConfiguration = client.bindArgs(sliderClientConfiguration,
+          new String[] { "help" });
+      client.init(sliderClientConfiguration);
+      client.start();
+    } catch (Exception e) {
+      logger.error("Unable to create SliderClient", e);
+      throw new RuntimeException(e.getMessage(), e);
+    }
+    return client;
+  }
+
+  private static String getUserToRunAs() {
+    String user = System.getenv(PROPERTY_APP_RUNAS_USER);
+    if (StringUtils.isEmpty(user)) {
+      user = "root";
+    }
+    return user;
+  }
+
+  private static org.apache.hadoop.conf.Configuration getSliderClientConfiguration() {
+    if (SLIDER_CONFIG != null) {
+      return SLIDER_CONFIG;
+    }
+    YarnConfiguration yarnConfig = new YarnConfiguration();
+    logger.info("prop yarn.resourcemanager.address = {}",
+        yarnConfig.get("yarn.resourcemanager.address"));
+
+    return yarnConfig;
+  }
+
+  private interface SliderClientContextRunnable<T> {
+    T run(SliderClient sliderClient)
+        throws YarnException, IOException, InterruptedException;
+  }
+
+  @GET
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON })
+  public Response getApplications(@QueryParam("state") String state) {
+    logger.info("GET: getApplications with param state = {}", state);
+
+    // Get all applications in a specific state - lighter projection. For full
+    // detail, call getApplication on a specific app.
+    Set<String> applications;
+    try {
+      if (StringUtils.isNotEmpty(state)) {
+        ApplicationStatus appStatus = new ApplicationStatus();
+        try {
+          ApplicationState.valueOf(state);
+        } catch (IllegalArgumentException e) {
+          appStatus.setDiagnostics("Invalid value for param state - " + state);
+          return Response.status(Status.BAD_REQUEST).entity(appStatus).build();
+        }
+        applications = getSliderApplications(state);
+      } else {
+        applications = getSliderApplications(true);
+      }
+    } catch (Exception e) {
+      logger.error("Get applications failed", e);
+      return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+    }
+
+    Set<Application> apps = new HashSet<Application>();
+    if (applications.size() > 0) {
+      try {
+        for (String app : applications) {
+          Application application = new Application();
+          // TODO: Need to get lifetime, launch-time and privileged container
+          // status from YARN
+          application.setLifetime(null);
+          application.setLaunchTime(new Date());
+          application.setName(app);
+          // Containers not required, setting to null to avoid empty list
+          application.setContainers(null);
+          apps.add(application);
+        }
+      } catch (Exception e) {
+        logger.error("Get applications failed", e);
+        return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+      }
+    }
+
+    return Response.ok().entity(apps).build();
+  }
+
+  @GET
+  @Path("/{app_name}")
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON })
+  public Response getApplication(@PathParam("app_name") String appName) {
+    logger.info("GET: getApplication for appName = {}", appName);
+
+    // app name validation
+    if (!SliderUtils.isClusternameValid(appName)) {
+      ApplicationStatus applicationStatus = new ApplicationStatus();
+      applicationStatus.setDiagnostics("Invalid application name");
+      applicationStatus.setCode(ERROR_CODE_APP_NAME_INVALID);
+      return Response.status(Status.NOT_FOUND).entity(applicationStatus)
+          .build();
+    }
+
+    // Check if app exists
+    try {
+      int livenessCheck = getSliderList(appName);
+      if (livenessCheck < 0) {
+        logger.info("Application not running");
+        ApplicationStatus applicationStatus = new ApplicationStatus();
+        applicationStatus.setDiagnostics(ERROR_APPLICATION_NOT_RUNNING);
+        applicationStatus.setCode(ERROR_CODE_APP_IS_NOT_RUNNING);
+        return Response.status(Status.NOT_FOUND).entity(applicationStatus)
+            .build();
+      }
+    } catch (UnknownApplicationInstanceException e) {
+      logger.error("Get application failed, application not found", e);
+      ApplicationStatus applicationStatus = new ApplicationStatus();
+      applicationStatus.setDiagnostics(ERROR_APPLICATION_DOES_NOT_EXIST);
+      applicationStatus.setCode(ERROR_CODE_APP_DOES_NOT_EXIST);
+      return Response.status(Status.NOT_FOUND).entity(applicationStatus)
+          .build();
+    } catch (Exception e) {
+      logger.error("Get application failed, application not running", e);
+      ApplicationStatus applicationStatus = new ApplicationStatus();
+      applicationStatus.setDiagnostics(ERROR_APPLICATION_NOT_RUNNING);
+      applicationStatus.setCode(ERROR_CODE_APP_IS_NOT_RUNNING);
+      return Response.status(Status.NOT_FOUND).entity(applicationStatus)
+          .build();
+    }
+
+    Application app = new Application();
+    app.setName(appName);
+    app.setUri(CONTEXT_ROOT + APPLICATIONS_API_RESOURCE_PATH + "/"
+        + appName);
+    // TODO: add status
+    app.setState(ApplicationState.ACCEPTED);
+    JsonObject appStatus = null;
+    JsonObject appRegistryDocker = null;
+    JsonObject appRegistryQuicklinks = null;
+    try {
+      appStatus = getSliderApplicationStatus(appName);
+      appRegistryDocker = getSliderApplicationRegistry(appName, "docker");
+      appRegistryQuicklinks = getSliderApplicationRegistry(appName,
+          "quicklinks");
+      return populateAppData(app, appStatus, appRegistryDocker,
+          appRegistryQuicklinks);
+    } catch (BadClusterStateException | NotFoundException e) {
+      logger.error(
+          "Get application failed, application not in running state yet", e);
+      ApplicationStatus applicationStatus = new ApplicationStatus();
+      applicationStatus.setDiagnostics("Application not running yet");
+      applicationStatus.setCode(ERROR_CODE_APP_SUBMITTED_BUT_NOT_RUNNING_YET);
+      return Response.status(Status.NOT_FOUND).entity(applicationStatus)
+          .build();
+    } catch (Exception e) {
+      logger.error("Get application failed", e);
+      ApplicationStatus applicationStatus = new ApplicationStatus();
+      applicationStatus.setDiagnostics("Failed to retrieve application: "
+          + e.getMessage());
+      return Response.status(Status.INTERNAL_SERVER_ERROR)
+          .entity(applicationStatus).build();
+    }
+  }
+
+  private Response populateAppData(Application app, JsonObject appStatus,
+      JsonObject appRegistryDocker, JsonObject appRegistryQuicklinks) {
+    String appName = jsonGetAsString(appStatus, "name");
+    Long totalNumberOfRunningContainers = 0L;
+    Long totalExpectedNumberOfRunningContainers = 0L;
+    Long totalNumberOfIpAssignedContainers = 0L;
+
+    // info
+    JsonObject applicationInfo = jsonGetAsObject(appStatus, "info");
+    if (applicationInfo != null) {
+      String applicationId = jsonGetAsString(applicationInfo, "info.am.app.id");
+      if (applicationId != null) {
+        app.setId(applicationId);
+      }
+    }
+
+    // state
+    String appState = jsonGetAsString(appStatus, "state");
+    switch (Integer.parseInt(appState)) {
+      case StateValues.STATE_LIVE:
+        app.setState(ApplicationState.STARTED);
+        break;
+      case StateValues.STATE_CREATED:
+      case StateValues.STATE_INCOMPLETE:
+      case StateValues.STATE_SUBMITTED:
+        app.setState(ApplicationState.ACCEPTED);
+        return Response.ok(app).build();
+      case StateValues.STATE_DESTROYED:
+      case StateValues.STATE_STOPPED:
+        app.setState(ApplicationState.STOPPED);
+        return Response.ok(app).build();
+      default:
+        break;
+    }
+
+    // start time
+    app.setLaunchTime(appStatus.get("createTime") == null ? null
+        : new Date(appStatus.get("createTime").getAsLong()));
+
+    // lifetime - set it to unlimited for now
+    // TODO: Once YARN-3813 and YARN-4205 are available - get it from YARN
+    app.setLifetime(DEFAULT_UNLIMITED_LIFETIME);
+
+    // Quicklinks
+    Map<String, String> appQuicklinks = new HashMap<>();
+    for (Map.Entry<String, JsonElement> quicklink : appRegistryQuicklinks
+        .entrySet()) {
+      appQuicklinks.put(quicklink.getKey(), quicklink.getValue() == null ? null
+          : quicklink.getValue().getAsString());
+    }
+    if (!appQuicklinks.isEmpty()) {
+      app.setQuicklinks(appQuicklinks);
+    }
+
+    ArrayList<String> componentNames = new ArrayList<>();
+
+    // status.live
+    JsonObject applicationStatus = jsonGetAsObject(appStatus, "status");
+    // roles
+    JsonObject applicationRoles = jsonGetAsObject(appStatus, "roles");
+    // statistics
+    JsonObject applicationStatistics = jsonGetAsObject(appStatus, "statistics");
+    if (applicationRoles == null) {
+      // initialize to empty object to avoid too many null checks
+      applicationRoles = new JsonObject();
+    }
+    if (applicationStatus != null) {
+      JsonObject applicationLive = jsonGetAsObject(applicationStatus, "live");
+      if (applicationLive != null) {
+        for (Entry<String, JsonElement> entry : applicationLive.entrySet()) {
+          if (entry.getKey().equals(SLIDER_APPMASTER_COMPONENT_NAME)) {
+            continue;
+          }
+          componentNames.add(entry.getKey());
+          JsonObject componentRole = applicationRoles.get(entry.getKey()) == null ? new JsonObject()
+              : applicationRoles.get(entry.getKey()).getAsJsonObject();
+          JsonObject liveContainers = entry.getValue().getAsJsonObject();
+          if (liveContainers != null) {
+            for (Map.Entry<String, JsonElement> liveContainerEntry : liveContainers
+                .entrySet()) {
+              String containerId = liveContainerEntry.getKey();
+              Container container = new Container();
+              container.setId(containerId);
+              JsonObject liveContainer = (JsonObject) liveContainerEntry
+                  .getValue();
+              container
+                  .setLaunchTime(liveContainer.get("startTime") == null ? null
+                      : new Date(liveContainer.get("startTime").getAsLong()));
+              container
+                  .setComponentName(jsonGetAsString(liveContainer, "role"));
+              container.setIp(jsonGetAsString(liveContainer, "ip"));
+              // If ip is non-null increment count
+              if (container.getIp() != null) {
+                totalNumberOfIpAssignedContainers++;
+              }
+              container.setHostname(jsonGetAsString(liveContainer, "hostname"));
+              container.setState(ContainerState.INIT);
+              if (StringUtils.isNotEmpty(container.getIp())
+                  && StringUtils.isNotEmpty(container.getHostname())) {
+                container.setState(ContainerState.READY);
+              }
+              container.setBareHost(jsonGetAsString(liveContainer, "host"));
+              container.setUri(CONTEXT_ROOT + APPLICATIONS_API_RESOURCE_PATH
+                  + "/" + appName + CONTAINERS_API_RESOURCE_PATH + "/"
+                  + containerId);
+              Resource resource = new Resource();
+              resource.setCpus(jsonGetAsInt(componentRole, "yarn.vcores"));
+              resource.setMemory(jsonGetAsString(componentRole, "yarn.memory"));
+              container.setResource(resource);
+              // TODO: add container property - for response only?
+              app.addContainer(container);
+            }
+          }
+        }
+      }
+    }
+
+    // application info
+    if (applicationRoles != null && !componentNames.isEmpty()) {
+      JsonObject applicationRole = jsonGetAsObject(applicationRoles,
+          componentNames.get(0));
+      if (applicationRole != null) {
+        Artifact artifact = new Artifact();
+        // how to get artifact id - docker image name??
+        artifact.setId(null);
+      }
+    }
+
+    // actual and expected number of containers
+    if (applicationStatistics != null) {
+      for (Entry<String, JsonElement> entry : applicationStatistics.entrySet()) {
+        if (entry.getKey().equals(SLIDER_APPMASTER_COMPONENT_NAME)) {
+          continue;
+        }
+        JsonObject containerStats = (JsonObject) entry.getValue();
+        totalNumberOfRunningContainers += jsonGetAsInt(containerStats,
+            "containers.live");
+        totalExpectedNumberOfRunningContainers += jsonGetAsInt(containerStats,
+            "containers.desired");
+      }
+      app.setNumberOfContainers(totalExpectedNumberOfRunningContainers);
+      app.setNumberOfRunningContainers(totalNumberOfRunningContainers);
+    }
+
+    // If all containers of the app has IP assigned, then according to the REST
+    // API it is considered to be READY. Note, application readiness from
+    // end-users point of view, is out of scope of the REST API. Also, this
+    // readiness has nothing to do with readiness-check defined at the component
+    // level (which is used for dependency resolution of component DAG).
+    if (totalNumberOfIpAssignedContainers == totalExpectedNumberOfRunningContainers) {
+      app.setState(ApplicationState.READY);
+    }
+    logger.info("Application = {}", app);
+    return Response.ok(app).build();
+  }
+
+  private String jsonGetAsString(JsonObject object, String key) {
+    return object.get(key) == null ? null : object.get(key).getAsString();
+  }
+
+  private Integer jsonGetAsInt(JsonObject object, String key) {
+    return object.get(key) == null ? null
+        : object.get(key).isJsonNull() ? null : object.get(key).getAsInt();
+  }
+
+  private JsonObject jsonGetAsObject(JsonObject object, String key) {
+    return object.get(key) == null ? null : object.get(key).getAsJsonObject();
+  }
+
+  private JsonObject getSliderApplicationStatus(final String appName)
+      throws IOException, YarnException, InterruptedException {
+    final File appStatusOutputFile = File.createTempFile("status_", ".json");
+    final ActionStatusArgs statusArgs = new ActionStatusArgs();
+    statusArgs.output = appStatusOutputFile.getAbsolutePath();
+
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<JsonObject>() {
+      @Override
+      public JsonObject run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        sliderClient.actionStatus(appName, statusArgs);
+        JsonParser parser = new JsonParser();
+        FileReader reader = null;
+        JsonElement statusElement = null;
+        try {
+          reader = new FileReader(appStatusOutputFile);
+          statusElement = parser.parse(reader);
+        } finally {
+          if (reader != null) {
+            reader.close();
+          }
+          appStatusOutputFile.delete();
+        }
+        return (statusElement == null || statusElement instanceof JsonNull) ?
+            new JsonObject() : (JsonObject) statusElement;
+      }
+    });
+  }
+
+  private JsonObject getSliderApplicationRegistry(final String appName,
+      final String registryName) throws IOException, YarnException,
+      InterruptedException {
+    final File appRegistryOutputFile = File
+        .createTempFile("registry_", ".json");
+    final ActionRegistryArgs registryArgs = new ActionRegistryArgs();
+    registryArgs.out = appRegistryOutputFile;
+    registryArgs.name = appName;
+    registryArgs.getConf = registryName;
+    registryArgs.format = ConfigFormat.JSON.toString();
+
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<JsonObject>() {
+      @Override
+      public JsonObject run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        sliderClient.actionRegistry(registryArgs);
+        JsonParser parser = new JsonParser();
+        FileReader reader = null;
+        JsonElement registryElement = null;
+        try {
+          reader = new FileReader(appRegistryOutputFile);
+          registryElement = parser.parse(reader);
+        } catch (Throwable t) {
+          logger.error("Error reading file {}", appRegistryOutputFile);
+        } finally {
+          if (reader != null) {
+            reader.close();
+          }
+          appRegistryOutputFile.delete();
+        }
+        return (registryElement == null || registryElement instanceof JsonNull) ?
+            new JsonObject() : (JsonObject) registryElement;
+      }
+    });
+  }
+
+  private Integer getSliderList(final String appName)
+      throws IOException, YarnException, InterruptedException {
+    return getSliderList(appName, true);
+  }
+
+  private Integer getSliderList(final String appName, final boolean liveOnly)
+      throws IOException, YarnException, InterruptedException {
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<Integer>() {
+      @Override
+      public Integer run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        int status = 0;
+        if (liveOnly) {
+          status = sliderClient.actionList(appName);
+        } else {
+          ActionListArgs listArgs = new ActionListArgs();
+          status = sliderClient.actionList(appName, listArgs);
+        }
+        return status;
+      }
+    });
+  }
+
+  private Set<String> getSliderApplications(final String state)
+      throws IOException, YarnException, InterruptedException {
+    return getSliderApplications(false, state);
+  }
+
+  private Set<String> getSliderApplications(final boolean liveOnly)
+      throws IOException, YarnException, InterruptedException {
+    return getSliderApplications(liveOnly, null);
+  }
+
+  private Set<String> getSliderApplications(final boolean liveOnly,
+      final String state) throws IOException, YarnException,
+      InterruptedException {
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<Set<String>>() {
+      @Override
+      public Set<String> run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        Set<String> apps;
+        ActionListArgs listArgs = new ActionListArgs();
+        if (liveOnly) {
+          apps = sliderClient.getApplicationList(null);
+        } else if (StringUtils.isNotEmpty(state)) {
+          listArgs.state = state;
+          apps = sliderClient.getApplicationList(null, listArgs);
+        } else {
+          apps = sliderClient.getApplicationList(null, listArgs);
+        }
+        return apps;
+      }
+    });
+  }
+
+  @DELETE
+  @Path("/{app_name}")
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON })
+  public Response deleteApplication(@PathParam("app_name") String appName) {
+    logger.info("DELETE: deleteApplication for appName = {}", appName);
+
+    try {
+      Response stopResponse = stopSliderApplication(appName);
+      if (stopResponse.getStatus() == Status.INTERNAL_SERVER_ERROR
+          .getStatusCode()) {
+        return Response.status(Status.NOT_FOUND).build();
+      }
+    } catch (UnknownApplicationInstanceException e) {
+      logger.error("Application does not exist", e);
+      return Response.status(Status.NOT_FOUND).build();
+    } catch (Exception e) {
+      logger.error("Delete application failed", e);
+      return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+    }
+
+    // Although slider client stop returns immediately, it usually takes a
+    // little longer for it to stop from YARN point of view. Slider destroy
+    // fails if the application is not completely stopped. Hence the need to
+    // call destroy in a controlled loop few times (only if exit code is
+    // EXIT_APPLICATION_IN_USE), before giving up.
+    boolean keepTrying = true;
+    int maxDeleteAttempt = 5;
+    int deleteAttempt = 0;
+    while (keepTrying && deleteAttempt < maxDeleteAttempt) {
+      try {
+        destroySliderApplication(appName);
+        keepTrying = false;
+      } catch (SliderException e) {
+        logger.error("Delete application threw exception", e);
+        if (e.getExitCode() == SliderExitCodes.EXIT_APPLICATION_IN_USE) {
+          deleteAttempt++;
+          try {
+            Thread.sleep(500);
+          } catch (InterruptedException e1) {
+          }
+        } else {
+          return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+        }
+      } catch (Exception e) {
+        logger.error("Delete application failed", e);
+        return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+      }
+    }
+    return Response.status(Status.NO_CONTENT).build();
+  }
+
+  private Response stopSliderApplication(final String appName)
+      throws IOException, YarnException, InterruptedException {
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<Response>() {
+      @Override
+      public Response run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        ActionFreezeArgs freezeArgs = new ActionFreezeArgs();
+        int returnCode = sliderClient.actionFreeze(appName, freezeArgs);
+        if (returnCode == 0) {
+          logger.info("Successfully stopped application {}", appName);
+          return Response.status(Status.NO_CONTENT).build();
+        } else {
+          logger.error("Stop of application {} failed with return code ",
+              appName, returnCode);
+          ApplicationStatus applicationStatus = new ApplicationStatus();
+          applicationStatus.setDiagnostics("Stop of application " + appName
+              + " failed");
+          return Response.status(Status.INTERNAL_SERVER_ERROR)
+              .entity(applicationStatus).build();
+        }
+      }
+    });
+  }
+
+  private Response startSliderApplication(final String appName)
+      throws IOException, YarnException, InterruptedException {
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<Response>() {
+      @Override
+      public Response run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        ActionThawArgs thawArgs = new ActionThawArgs();
+        int returnCode = sliderClient.actionThaw(appName, thawArgs);
+        if (returnCode == 0) {
+          logger.info("Successfully started application {}", appName);
+          ApplicationStatus applicationStatus = new ApplicationStatus();
+          applicationStatus.setState(ApplicationState.ACCEPTED);
+          applicationStatus.setUri(CONTEXT_ROOT
+              + APPLICATIONS_API_RESOURCE_PATH + "/" + appName);
+          // 202 = ACCEPTED
+          return Response.status(HTTP_STATUS_CODE_ACCEPTED)
+              .entity(applicationStatus).build();
+        } else {
+          logger.error("Start of application {} failed with returnCode ",
+              appName, returnCode);
+          ApplicationStatus applicationStatus = new ApplicationStatus();
+          applicationStatus.setDiagnostics("Start of application " + appName
+              + " failed");
+          return Response.status(Status.INTERNAL_SERVER_ERROR)
+              .entity(applicationStatus).build();
+        }
+      }
+    });
+  }
+
+  private Void destroySliderApplication(final String appName)
+      throws IOException, YarnException, InterruptedException {
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<Void>() {
+      @Override
+      public Void run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        sliderClient.actionDestroy(appName);
+        return null;
+      }
+    });
+  }
+
+  @PUT
+  @Path("/{app_name}")
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON })
+  public Response updateApplication(@PathParam("app_name") String appName,
+      Application updateAppData) {
+    logger.info("PUT: updateApplication for app = {} with data = {}", appName,
+        updateAppData);
+
+    // Ignore the app name provided in updateAppData and always use appName
+    // path param
+    updateAppData.setName(appName);
+
+    // Adding support for stop and start
+    // For STOP the app should be running. If already stopped then this
+    // operation will be a no-op. For START it should be in stopped state.
+    // If already running then this operation will be a no-op.
+
+    // Check if app exists in any state
+    try {
+      int appsFound = getSliderList(appName, false);
+      if (appsFound < 0) {
+        return Response.status(Status.NOT_FOUND).build();
+      }
+    } catch (Exception e) {
+      logger.error("Update application failed", e);
+      return Response.status(Status.NOT_FOUND).build();
+    }
+
+    // If a STOP is requested
+    if (updateAppData.getState() != null
+        && updateAppData.getState() == ApplicationState.STOPPED) {
+      try {
+        int livenessCheck = getSliderList(appName);
+        if (livenessCheck == 0) {
+          return stopSliderApplication(appName);
+        } else {
+          logger.info("Application {} is already stopped", appName);
+          ApplicationStatus applicationStatus = new ApplicationStatus();
+          applicationStatus.setDiagnostics("Application " + appName
+              + " is already stopped");
+          return Response.status(Status.BAD_REQUEST).entity(applicationStatus)
+              .build();
+        }
+      } catch (Exception e) {
+        logger.error("Stop application failed", e);
+        return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+      }
+    }
+
+    // If a START is requested
+    if (updateAppData.getState() != null
+        && updateAppData.getState() == ApplicationState.STARTED) {
+      try {
+        int livenessCheck = getSliderList(appName);
+        if (livenessCheck != 0) {
+          return startSliderApplication(appName);
+        } else {
+          logger.info("Application {} is already running", appName);
+          ApplicationStatus applicationStatus = new ApplicationStatus();
+          applicationStatus.setDiagnostics("Application " + appName
+              + " is already running");
+          applicationStatus.setUri(CONTEXT_ROOT
+              + APPLICATIONS_API_RESOURCE_PATH + "/" + appName);
+          return Response.status(Status.BAD_REQUEST).entity(applicationStatus)
+              .build();
+        }
+      } catch (Exception e) {
+        logger.error("Start application failed", e);
+        return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+      }
+    }
+
+    // If no of instances specified then treat it as a flex
+    if (updateAppData.getNumberOfContainers() != null
+        && updateAppData.getComponents() == null) {
+      updateAppData.setComponents(getDefaultComponentAsList());
+    }
+
+    // At this point if there are components then it is a flex
+    if (updateAppData.getComponents() != null) {
+      try {
+        int livenessCheck = getSliderList(appName);
+        if (livenessCheck == 0) {
+          flexSliderApplication(appName, updateAppData);
+        }
+        return Response.status(Status.NO_CONTENT).build();
+      } catch (Exception e) {
+        logger.error("Update application failed", e);
+        return Response.status(Status.INTERNAL_SERVER_ERROR).build();
+      }
+    }
+
+    // If new lifetime value specified then update it
+    if (updateAppData.getLifetime() != null) {
+      // TODO: Once YARN-3813 and YARN-4205 are available
+    }
+
+    // If nothing happens consider it a no-op
+    return Response.status(Status.NO_CONTENT).build();
+  }
+
+  private List<Component> getDefaultComponentAsList() {
+    Component comp = new Component();
+    comp.setName(DEFAULT_COMPONENT_NAME);
+    List<Component> comps = new ArrayList<>();
+    comps.add(comp);
+    return comps;
+  }
+
+  private Void flexSliderApplication(final String appName,
+      final Application updateAppData) throws IOException, YarnException,
+      InterruptedException {
+    return invokeSliderClientRunnable(new SliderClientContextRunnable<Void>() {
+      @Override
+      public Void run(SliderClient sliderClient) throws YarnException,
+          IOException, InterruptedException {
+        ActionFlexArgs flexArgs = new ActionFlexArgs();
+        ComponentArgsDelegate compDelegate = new ComponentArgsDelegate();
+        Long globalNumberOfContainers = updateAppData.getNumberOfContainers();
+        for (Component comp : updateAppData.getComponents()) {
+          Long noOfContainers = comp.getNumberOfContainers() == null
+              ? globalNumberOfContainers : comp.getNumberOfContainers();
+          if (noOfContainers != null) {
+            compDelegate.componentTuples.addAll(
+                Arrays.asList(comp.getName(), String.valueOf(noOfContainers)));
+          }
+        }
+        if (!compDelegate.componentTuples.isEmpty()) {
+          flexArgs.componentDelegate = compDelegate;
+          sliderClient.actionFlex(appName, flexArgs);
+        }
+        return null;
+      }
+    });
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org