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 ey...@apache.org on 2017/12/14 00:29:16 UTC

hadoop git commit: YARN-7540. Route YARN service CLI function through YARN Service API. (Contributed by Eric Yang)

Repository: hadoop
Updated Branches:
  refs/heads/trunk 46e18c8da -> 438c1d333


YARN-7540.  Route YARN service CLI function through YARN Service API. (Contributed by Eric Yang)


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

Branch: refs/heads/trunk
Commit: 438c1d333ebc0a3071bb556532ed959a4bd1e6d6
Parents: 46e18c8
Author: Eric Yang <ey...@apache.org>
Authored: Wed Dec 13 19:29:04 2017 -0500
Committer: Eric Yang <ey...@apache.org>
Committed: Wed Dec 13 19:29:04 2017 -0500

----------------------------------------------------------------------
 .../hadoop-yarn-services-api/pom.xml            |   9 +
 .../yarn/service/client/ApiServiceClient.java   | 430 +++++++++++++++++++
 .../hadoop/yarn/service/webapp/ApiServer.java   |  34 +-
 .../yarn/service/webapp/package-info.java       |  28 ++
 .../service/client/TestApiServiceClient.java    | 259 +++++++++++
 .../src/test/resources/example-app.json         |  15 +
 .../src/test/resources/log4j.properties         |  19 +
 .../service/api/records/ReadinessCheck.java     |   2 +
 .../yarn/service/api/records/Resource.java      |   6 +-
 .../yarn/service/api/records/ServiceState.java  |   2 +-
 .../yarn/service/component/Component.java       |   2 +-
 .../hadoop/yarn/service/utils/JsonSerDeser.java |   3 +
 .../client/TestBuildExternalComponents.java     |   2 +-
 .../yarn/service/client/TestServiceCLI.java     |  14 +-
 .../hadoop/yarn/client/api/AppAdminClient.java  |   6 +
 15 files changed, 818 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/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
index ddea2a1..bae62c6 100644
--- 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
@@ -65,6 +65,15 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.rat</groupId>
+        <artifactId>apache-rat-plugin</artifactId>
+        <configuration>
+          <excludes>
+            <exclude>**/*.json</exclude>
+          </excludes>
+        </configuration>
+      </plugin>
     </plugins>
   </build>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
new file mode 100644
index 0000000..f4133a5
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/ApiServiceClient.java
@@ -0,0 +1,430 @@
+/*
+ * 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.service.client;
+
+import static org.apache.hadoop.yarn.service.utils.ServiceApiUtil.jsonSerDeser;
+
+import java.io.File;
+import java.io.IOException;
+import java.text.MessageFormat;
+import java.util.List;
+import java.util.Map;
+
+import javax.ws.rs.core.MediaType;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.client.api.AppAdminClient;
+import org.apache.hadoop.yarn.client.api.YarnClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.service.api.records.Component;
+import org.apache.hadoop.yarn.service.api.records.Service;
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
+import org.apache.hadoop.yarn.service.api.records.ServiceStatus;
+import org.apache.hadoop.yarn.util.RMHAUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.sun.jersey.api.client.Client;
+import com.sun.jersey.api.client.ClientResponse;
+import com.sun.jersey.api.client.WebResource;
+import com.sun.jersey.api.client.WebResource.Builder;
+import com.sun.jersey.api.client.config.ClientConfig;
+import com.sun.jersey.api.client.config.DefaultClientConfig;
+
+import static org.apache.hadoop.yarn.service.exceptions.LauncherExitCodes.*;
+
+/**
+ * The rest API client for users to manage services on YARN.
+ */
+public class ApiServiceClient extends AppAdminClient {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ApiServiceClient.class);
+  protected YarnClient yarnClient;
+
+  @Override protected void serviceInit(Configuration configuration)
+      throws Exception {
+    yarnClient = YarnClient.createYarnClient();
+    addService(yarnClient);
+    super.serviceInit(configuration);
+  }
+
+  /**
+   * Calculate Resource Manager address base on working REST API.
+   */
+  private String getRMWebAddress() {
+    Configuration conf = getConfig();
+    String scheme = "http://";
+    String path = "/app/v1/services/version";
+    String rmAddress = conf
+        .get("yarn.resourcemanager.webapp.address");
+    if(conf.getBoolean("hadoop.ssl.enabled", false)) {
+      scheme = "https://";
+      rmAddress = conf
+          .get("yarn.resourcemanager.webapp.https.address");
+    }
+
+    List<String> rmServers = RMHAUtils
+        .getRMHAWebappAddresses(new YarnConfiguration(conf));
+    for (String host : rmServers) {
+      StringBuilder sb = new StringBuilder();
+      sb.append(scheme);
+      sb.append(host);
+      sb.append(path);
+      Client client = Client.create();
+      WebResource webResource = client
+          .resource(sb.toString());
+      String test = webResource.get(String.class);
+      if (test.contains("hadoop_version")) {
+        rmAddress = host;
+        break;
+      }
+    }
+    return scheme+rmAddress;
+  }
+
+  /**
+   * Compute active resource manager API service location.
+   *
+   * @param appName - YARN service name
+   * @return URI to API Service
+   * @throws IOException
+   */
+  private String getApiUrl(String appName) throws IOException {
+    String url = getRMWebAddress();
+    StringBuilder api = new StringBuilder();
+    api.append(url);
+    api.append("/app/v1/services");
+    if (appName != null) {
+      api.append("/");
+      api.append(appName);
+    }
+    return api.toString();
+  }
+
+  private Builder getApiClient() throws IOException {
+    return getApiClient(null);
+  }
+
+  /**
+   * Setup API service web request.
+   *
+   * @param appName
+   * @return
+   * @throws IOException
+   */
+  private Builder getApiClient(String appName) throws IOException {
+    Client client = Client.create(getClientConfig());
+    Configuration conf = getConfig();
+    client.setChunkedEncodingSize(null);
+    Builder builder = client
+        .resource(getApiUrl(appName)).type(MediaType.APPLICATION_JSON);
+    if (conf.get("hadoop.security.authentication").equals("kerberos")) {
+      AuthenticatedURL.Token token = new AuthenticatedURL.Token();
+      builder.header("WWW-Authenticate", token);
+    }
+    return builder
+        .accept("application/json;charset=utf-8");
+  }
+
+  private ClientConfig getClientConfig() {
+    ClientConfig config = new DefaultClientConfig();
+    config.getProperties().put(
+        ClientConfig.PROPERTY_CHUNKED_ENCODING_SIZE, 0);
+    config.getProperties().put(
+        ClientConfig.PROPERTY_BUFFER_RESPONSE_ENTITY_ON_EXCEPTION, true);
+    return config;
+  }
+
+  private int processResponse(ClientResponse response) {
+    response.bufferEntity();
+    if (response.getStatus() >= 299) {
+      String error = "";
+      try {
+        ServiceStatus ss = response.getEntity(ServiceStatus.class);
+        error = ss.getDiagnostics();
+      } catch (Throwable t) {
+        error = response.getEntity(String.class);
+      }
+      LOG.error(error);
+      return EXIT_EXCEPTION_THROWN;
+    }
+    LOG.info(response.toString());
+    return EXIT_SUCCESS;
+  }
+
+  /**
+   * Utility method to load Service json from disk or from
+   * YARN examples.
+   *
+   * @param fileName - path to yarnfile
+   * @param serviceName - YARN Service Name
+   * @param lifetime - application lifetime
+   * @param queue - Queue to submit application
+   * @return
+   * @throws IOException
+   * @throws YarnException
+   */
+  public Service loadAppJsonFromLocalFS(String fileName, String serviceName,
+      Long lifetime, String queue) throws IOException, YarnException {
+    File file = new File(fileName);
+    if (!file.exists() && fileName.equals(file.getName())) {
+      String examplesDirStr = System.getenv("YARN_SERVICE_EXAMPLES_DIR");
+      String[] examplesDirs;
+      if (examplesDirStr == null) {
+        String yarnHome = System
+            .getenv(ApplicationConstants.Environment.HADOOP_YARN_HOME.key());
+        examplesDirs = new String[]{
+            yarnHome + "/share/hadoop/yarn/yarn-service-examples",
+            yarnHome + "/yarn-service-examples"
+        };
+      } else {
+        examplesDirs = StringUtils.split(examplesDirStr, ":");
+      }
+      for (String dir : examplesDirs) {
+        file = new File(MessageFormat.format("{0}/{1}/{2}.json",
+            dir, fileName, fileName));
+        if (file.exists()) {
+          break;
+        }
+        // Then look for secondary location.
+        file = new File(MessageFormat.format("{0}/{1}.json",
+            dir, fileName));
+        if (file.exists()) {
+          break;
+        }
+      }
+    }
+    if (!file.exists()) {
+      throw new YarnException("File or example could not be found: " +
+          fileName);
+    }
+    Path filePath = new Path(file.getAbsolutePath());
+    LOG.info("Loading service definition from local FS: " + filePath);
+    Service service = jsonSerDeser
+        .load(FileSystem.getLocal(getConfig()), filePath);
+    if (!StringUtils.isEmpty(serviceName)) {
+      service.setName(serviceName);
+    }
+    if (lifetime != null && lifetime > 0) {
+      service.setLifetime(lifetime);
+    }
+    if (!StringUtils.isEmpty(queue)) {
+      service.setQueue(queue);
+    }
+    return service;
+  }
+
+  /**
+   * Launch YARN service application.
+   *
+   * @param fileName - path to yarnfile
+   * @param appName - YARN Service Name
+   * @param lifetime - application lifetime
+   * @param queue - Queue to submit application
+   */
+  @Override
+  public int actionLaunch(String fileName, String appName, Long lifetime,
+      String queue) throws IOException, YarnException {
+    int result = EXIT_SUCCESS;
+    try {
+      Service service =
+          loadAppJsonFromLocalFS(fileName, appName, lifetime, queue);
+      String buffer = jsonSerDeser.toJson(service);
+      ClientResponse response = getApiClient()
+          .post(ClientResponse.class, buffer);
+      result = processResponse(response);
+    } catch (Exception e) {
+      LOG.error("Fail to launch application: ", e);
+      result = EXIT_EXCEPTION_THROWN;
+    }
+    return result;
+  }
+
+  /**
+   * Stop YARN service application.
+   *
+   * @param appName - YARN Service Name
+   */
+  @Override
+  public int actionStop(String appName) throws IOException, YarnException {
+    int result = EXIT_SUCCESS;
+    try {
+      Service service = new Service();
+      service.setName(appName);
+      service.setState(ServiceState.STOPPED);
+      String buffer = jsonSerDeser.toJson(service);
+      ClientResponse response = getApiClient(appName)
+          .put(ClientResponse.class, buffer);
+      result = processResponse(response);
+    } catch (Exception e) {
+      LOG.error("Fail to stop application: ", e);
+      result = EXIT_EXCEPTION_THROWN;
+    }
+    return result;
+  }
+
+  /**
+   * Start YARN service application.
+   *
+   * @param appName - YARN Service Name
+   */
+  @Override
+  public int actionStart(String appName) throws IOException, YarnException {
+    int result = EXIT_SUCCESS;
+    try {
+      Service service = new Service();
+      service.setName(appName);
+      service.setState(ServiceState.STARTED);
+      String buffer = jsonSerDeser.toJson(service);
+      ClientResponse response = getApiClient(appName)
+          .put(ClientResponse.class, buffer);
+      result = processResponse(response);
+    } catch (Exception e) {
+      LOG.error("Fail to start application: ", e);
+      result = EXIT_EXCEPTION_THROWN;
+    }
+    return result;
+  }
+
+  /**
+   * Save Service configuration.
+   *
+   * @param fileName - path to Yarnfile
+   * @param appName - YARN Service Name
+   * @param lifetime - container life time
+   * @param queue - Queue to submit the application
+   */
+  @Override
+  public int actionSave(String fileName, String appName, Long lifetime,
+      String queue) throws IOException, YarnException {
+    int result = EXIT_SUCCESS;
+    try {
+      Service service =
+          loadAppJsonFromLocalFS(fileName, appName, lifetime, queue);
+      service.setState(ServiceState.STOPPED);
+      String buffer = jsonSerDeser.toJson(service);
+      ClientResponse response = getApiClient()
+          .post(ClientResponse.class, buffer);
+      result = processResponse(response);
+    } catch (Exception e) {
+      LOG.error("Fail to save application: ", e);
+      result = EXIT_EXCEPTION_THROWN;
+    }
+    return result;
+  }
+
+  /**
+   * Decommission a YARN service.
+   *
+   * @param appName - YARN Service Name
+   */
+  @Override
+  public int actionDestroy(String appName) throws IOException, YarnException {
+    int result = EXIT_SUCCESS;
+    try {
+      ClientResponse response = getApiClient(appName)
+          .delete(ClientResponse.class);
+      result = processResponse(response);
+    } catch (Exception e) {
+      LOG.error("Fail to destroy application: ", e);
+      result = EXIT_EXCEPTION_THROWN;
+    }
+    return result;
+  }
+
+  /**
+   * Change number of containers associated with a service.
+   *
+   * @param appName - YARN Service Name
+   * @param cmponentCounts - list of components and desired container count
+   */
+  @Override
+  public int actionFlex(String appName, Map<String, String> componentCounts)
+      throws IOException, YarnException {
+    int result = EXIT_SUCCESS;
+    try {
+      Service service = new Service();
+      service.setName(appName);
+      service.setState(ServiceState.FLEX);
+      for (Map.Entry<String, String> entry : componentCounts.entrySet()) {
+        Component component = new Component();
+        component.setName(entry.getKey());
+        Long numberOfContainers = Long.parseLong(entry.getValue());
+        component.setNumberOfContainers(numberOfContainers);
+        service.addComponent(component);
+      }
+      String buffer = jsonSerDeser.toJson(service);
+      ClientResponse response = getApiClient(appName)
+          .put(ClientResponse.class, buffer);
+      result = processResponse(response);
+    } catch (Exception e) {
+      LOG.error("Fail to flex application: ", e);
+      result = EXIT_EXCEPTION_THROWN;
+    }
+    return result;
+  }
+
+  @Override
+  public int enableFastLaunch() throws IOException, YarnException {
+    ServiceClient sc = new ServiceClient();
+    sc.init(getConfig());
+    sc.start();
+    int result = sc.enableFastLaunch();
+    sc.close();
+    return result;
+  }
+
+  /**
+   * Retrieve Service Status through REST API.
+   *
+   * @param applicationId - YARN application ID
+   * @return Status output
+   */
+  @Override
+  public String getStatusString(String applicationId) throws IOException,
+      YarnException {
+    String output = "";
+    try {
+      ApplicationReport appReport = yarnClient
+          .getApplicationReport(ApplicationId.fromString(applicationId));
+
+      String appName = appReport.getName();
+      ClientResponse response = getApiClient(appName)
+          .get(ClientResponse.class);
+      if (response.getStatus() != 200) {
+        StringBuilder sb = new StringBuilder();
+        sb.append(appName);
+        sb.append(" Failed : HTTP error code : ");
+        sb.append(response.getStatus());
+        return sb.toString();
+      }
+      output = response.getEntity(String.class);
+    } catch (Exception e) {
+      LOG.error("Fail to check application status: ", e);
+    }
+    return output;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
index 34ab8f0..84c3905 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.service.api.records.Service;
 import org.apache.hadoop.yarn.service.api.records.ServiceState;
 import org.apache.hadoop.yarn.service.api.records.ServiceStatus;
 import org.apache.hadoop.yarn.service.client.ServiceClient;
-import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,10 +46,12 @@ import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.Status;
 import java.io.IOException;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
 
 import static org.apache.hadoop.yarn.service.api.records.ServiceState.ACCEPTED;
 import static org.apache.hadoop.yarn.service.conf.RestApiConstants.*;
+import static org.apache.hadoop.yarn.service.exceptions.LauncherExitCodes.EXIT_SUCCESS;
 
 /**
  * The rest API endpoints for users to manage services on YARN.
@@ -102,9 +103,13 @@ public class ApiServer {
     LOG.info("POST: createService = {}", service);
     ServiceStatus serviceStatus = new ServiceStatus();
     try {
-      ApplicationId applicationId = SERVICE_CLIENT.actionCreate(service);
-      LOG.info("Successfully created service " + service.getName()
+      if(service.getState()==ServiceState.STOPPED) {
+        SERVICE_CLIENT.actionBuild(service);
+      } else {
+        ApplicationId applicationId = SERVICE_CLIENT.actionCreate(service);
+        LOG.info("Successfully created service " + service.getName()
           + " applicationId = " + applicationId);
+      }
       serviceStatus.setState(ACCEPTED);
       serviceStatus.setUri(
           CONTEXT_ROOT + SERVICE_ROOT_PATH + "/" + service
@@ -223,6 +228,29 @@ public class ApiServer {
     // path param
     updateServiceData.setName(appName);
 
+    if (updateServiceData.getState() == ServiceState.FLEX) {
+      Map<String, String> componentCountStrings = new HashMap<String, String>();
+      for (Component c : updateServiceData.getComponents()) {
+        componentCountStrings.put(c.getName(), c.getNumberOfContainers().toString());
+      }
+      ServiceStatus status = new ServiceStatus();
+      try {
+        int result = SERVICE_CLIENT
+            .actionFlex(appName, componentCountStrings);
+        if (result == EXIT_SUCCESS) {
+          LOG.info("Successfully flex service " + appName);
+          status.setDiagnostics("Service " + appName +
+              " is successfully flexed.");
+          status.setState(ServiceState.ACCEPTED);
+        }
+      } catch (YarnException | IOException e) {
+        String message = "Failed to flex service " + appName;
+        LOG.info(message, e);
+        status.setDiagnostics(message);
+        return Response.status(Status.INTERNAL_SERVER_ERROR)
+            .entity(status).build();
+      }
+    }
     // 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.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/package-info.java
new file mode 100644
index 0000000..1bdf05a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/package-info.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * Package org.apache.hadoop.yarn.service.webapp contains classes to be used
+ * for YARN Services API.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.service.webapp;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/client/TestApiServiceClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/client/TestApiServiceClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/client/TestApiServiceClient.java
new file mode 100644
index 0000000..ffd9328
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/java/org/apache/hadoop/yarn/service/client/TestApiServiceClient.java
@@ -0,0 +1,259 @@
+/*
+ * 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.service.client;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.HashMap;
+
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.eclipse.jetty.server.Server;
+import org.eclipse.jetty.server.ServerConnector;
+import org.eclipse.jetty.servlet.ServletContextHandler;
+import org.eclipse.jetty.servlet.ServletHolder;
+import org.eclipse.jetty.util.thread.QueuedThreadPool;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.apache.hadoop.yarn.service.exceptions.LauncherExitCodes.*;
+
+/**
+ * Test case for CLI to API Service.
+ *
+ */
+public class TestApiServiceClient {
+  private static ApiServiceClient asc;
+  private static ApiServiceClient badAsc;
+  private static Server server;
+
+  /**
+   * A mocked version of API Service for testing purpose.
+   *
+   */
+  @SuppressWarnings("serial")
+  public static class TestServlet extends HttpServlet {
+
+    @Override
+    protected void doGet(HttpServletRequest req, HttpServletResponse resp)
+        throws ServletException, IOException {
+      System.out.println("Get was called");
+      resp.setStatus(HttpServletResponse.SC_OK);
+    }
+
+    @Override
+    protected void doPost(HttpServletRequest req, HttpServletResponse resp)
+        throws ServletException, IOException {
+      resp.setStatus(HttpServletResponse.SC_OK);
+    }
+
+    @Override
+    protected void doPut(HttpServletRequest req, HttpServletResponse resp)
+        throws ServletException, IOException {
+      resp.setStatus(HttpServletResponse.SC_OK);
+    }
+
+    @Override
+    protected void doDelete(HttpServletRequest req, HttpServletResponse resp)
+        throws ServletException, IOException {
+      resp.setStatus(HttpServletResponse.SC_OK);
+    }
+
+  }
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    server = new Server(8088);
+    ((QueuedThreadPool)server.getThreadPool()).setMaxThreads(10);
+    ServletContextHandler context = new ServletContextHandler();
+    context.setContextPath("/app");
+    server.setHandler(context);
+    context.addServlet(new ServletHolder(TestServlet.class), "/*");
+    ((ServerConnector)server.getConnectors()[0]).setHost("localhost");
+    server.start();
+
+    Configuration conf = new Configuration();
+    conf.set("yarn.resourcemanager.webapp.address",
+        "localhost:8088");
+    asc = new ApiServiceClient();
+    asc.serviceInit(conf);
+
+    Configuration conf2 = new Configuration();
+    conf2.set("yarn.resourcemanager.webapp.address",
+        "localhost:8089");
+    badAsc = new ApiServiceClient();
+    badAsc.serviceInit(conf2);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    server.stop();
+  }
+
+  @Test
+  public void testLaunch() {
+    String fileName = "target/test-classes/example-app.json";
+    String appName = "example-app";
+    long lifetime = 3600L;
+    String queue = "default";
+    try {
+      int result = asc.actionLaunch(fileName, appName, lifetime, queue);
+      assertEquals(EXIT_SUCCESS, result);
+    } catch (IOException | YarnException e) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testBadLaunch() {
+    String fileName = "unknown_file";
+    String appName = "unknown_app";
+    long lifetime = 3600L;
+    String queue = "default";
+    try {
+      int result = badAsc.actionLaunch(fileName, appName, lifetime, queue);
+      assertEquals(EXIT_EXCEPTION_THROWN, result);
+    } catch (IOException | YarnException e) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testStop() {
+    String appName = "example-app";
+    try {
+      int result = asc.actionStop(appName);
+      assertEquals(EXIT_SUCCESS, result);
+    } catch (IOException | YarnException e) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testBadStop() {
+    String appName = "unknown_app";
+    try {
+      int result = badAsc.actionStop(appName);
+      assertEquals(EXIT_EXCEPTION_THROWN, result);
+    } catch (IOException | YarnException e) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testStart() {
+    String appName = "example-app";
+    try {
+      int result = asc.actionStart(appName);
+      assertEquals(EXIT_SUCCESS, result);
+    } catch (IOException | YarnException e) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testBadStart() {
+    String appName = "unknown_app";
+    try {
+      int result = badAsc.actionStart(appName);
+      assertEquals(EXIT_EXCEPTION_THROWN, result);
+    } catch (IOException | YarnException e) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testSave() {
+    String fileName = "target/test-classes/example-app.json";
+    String appName = "example-app";
+    long lifetime = 3600L;
+    String queue = "default";
+    try {
+      int result = asc.actionSave(fileName, appName, lifetime, queue);
+      assertEquals(EXIT_SUCCESS, result);
+    } catch (IOException | YarnException e) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testBadSave() {
+    String fileName = "unknown_file";
+    String appName = "unknown_app";
+    long lifetime = 3600L;
+    String queue = "default";
+    try {
+      int result = badAsc.actionSave(fileName, appName, lifetime, queue);
+      assertEquals(EXIT_EXCEPTION_THROWN, result);
+    } catch (IOException | YarnException e) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testFlex() {
+    String appName = "example-app";
+    HashMap<String, String> componentCounts = new HashMap<String, String>();
+    try {
+      int result = asc.actionFlex(appName, componentCounts);
+      assertEquals(EXIT_SUCCESS, result);
+    } catch (IOException | YarnException e) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testBadFlex() {
+    String appName = "unknown_app";
+    HashMap<String, String> componentCounts = new HashMap<String, String>();
+    try {
+      int result = badAsc.actionFlex(appName, componentCounts);
+      assertEquals(EXIT_EXCEPTION_THROWN, result);
+    } catch (IOException | YarnException e) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testDestroy() {
+    String appName = "example-app";
+    try {
+      int result = asc.actionDestroy(appName);
+      assertEquals(EXIT_SUCCESS, result);
+    } catch (IOException | YarnException e) {
+      fail();
+    }
+  }
+
+  @Test
+  public void testBadDestroy() {
+    String appName = "unknown_app";
+    try {
+      int result = badAsc.actionDestroy(appName);
+      assertEquals(EXIT_EXCEPTION_THROWN, result);
+    } catch (IOException | YarnException e) {
+      fail();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/resources/example-app.json
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/resources/example-app.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/resources/example-app.json
new file mode 100644
index 0000000..5dfbd64
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/resources/example-app.json
@@ -0,0 +1,15 @@
+{
+  "name": "example-app",
+  "components" :
+  [
+    {
+      "name": "simple",
+      "number_of_containers": 1,
+      "launch_command": "sleep 2",
+      "resource": {
+        "cpus": 1,
+        "memory": "128"
+      }
+    }
+  ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/resources/log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/resources/log4j.properties
new file mode 100644
index 0000000..81a3f6a
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services-api/src/test/resources/log4j.properties
@@ -0,0 +1,19 @@
+#   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.
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=info,stdout
+log4j.threshold=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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ReadinessCheck.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ReadinessCheck.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ReadinessCheck.java
index 2bcf68b..af7c542 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ReadinessCheck.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ReadinessCheck.java
@@ -32,6 +32,7 @@ import javax.xml.bind.annotation.XmlEnum;
 import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlType;
 
+import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonValue;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -49,6 +50,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
 @XmlRootElement
 @XmlAccessorType(XmlAccessType.FIELD)
+@JsonInclude(JsonInclude.Include.NON_NULL)
 public class ReadinessCheck implements Serializable {
   private static final long serialVersionUID = -3836839816887186801L;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Resource.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Resource.java
index dfdf92a..8f682b2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Resource.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Resource.java
@@ -22,7 +22,7 @@ import io.swagger.annotations.ApiModelProperty;
 
 import java.util.Objects;
 
-import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -104,8 +104,8 @@ public class Resource extends BaseResource implements Cloneable {
     this.memory = memory;
   }
 
-  @JsonIgnore
-  public long getMemoryMB() {
+  @JsonIgnoreProperties(ignoreUnknown=true)
+  public long calcMemoryMB() {
     if (this.memory == null) {
       return 0;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java
index d2f5d06..902a0b1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ServiceState.java
@@ -29,5 +29,5 @@ import org.apache.hadoop.classification.InterfaceStability;
 @ApiModel(description = "The current state of an service.")
 @javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
 public enum ServiceState {
-  ACCEPTED, STARTED, STABLE, STOPPED, FAILED;
+  ACCEPTED, STARTED, STABLE, STOPPED, FAILED, FLEX;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
index 5189ab1..9c5cbae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/component/Component.java
@@ -352,7 +352,7 @@ public class Component implements EventHandler<ComponentEvent> {
   @SuppressWarnings({ "unchecked" })
   public void requestContainers(long count) {
     Resource resource = Resource
-        .newInstance(componentSpec.getResource().getMemoryMB(),
+        .newInstance(componentSpec.getResource().calcMemoryMB(),
             componentSpec.getResource().getCpus());
 
     for (int i = 0; i < count; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/JsonSerDeser.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/JsonSerDeser.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/JsonSerDeser.java
index 7b22e3e..2c27ea7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/JsonSerDeser.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/utils/JsonSerDeser.java
@@ -60,10 +60,13 @@ public class JsonSerDeser<T> {
    * Create an instance bound to a specific type
    * @param classType class type
    */
+  @SuppressWarnings("deprecation")
   public JsonSerDeser(Class<T> classType) {
     this.classType = classType;
     this.mapper = new ObjectMapper();
     mapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+    mapper.configure(SerializationConfig.Feature.WRITE_NULL_MAP_VALUES, false);
+    mapper.configure(SerializationConfig.Feature.WRITE_NULL_PROPERTIES, false);
   }
 
   public JsonSerDeser(Class<T> classType, PropertyNamingStrategy namingStrategy) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestBuildExternalComponents.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestBuildExternalComponents.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestBuildExternalComponents.java
index 1f4581e..6d5bb20 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestBuildExternalComponents.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestBuildExternalComponents.java
@@ -60,7 +60,7 @@ public class TestBuildExternalComponents {
   private void buildAndCheckComponents(String appName, String appDef,
       SliderFileSystem sfs, Set<String> names) throws Throwable {
     AppAdminClient client = AppAdminClient.createAppAdminClient(AppAdminClient
-        .DEFAULT_TYPE, conf);
+        .UNIT_TEST_TYPE, conf);
     client.actionSave(ExampleAppJson.resourceName(appDef), null, null,
         null);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java
index df4b1df..a95818f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/client/TestServiceCLI.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.service.client;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.yarn.client.api.AppAdminClient;
 import org.apache.hadoop.yarn.client.cli.ApplicationCLI;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.service.api.records.Component;
@@ -61,15 +62,20 @@ public class TestServiceCLI {
   }
 
   private void buildApp(String serviceName, String appDef) throws Throwable {
-    String[] args = {"app", "-D", basedirProp, "-save", serviceName,
-        ExampleAppJson.resourceName(appDef)};
+    String[] args = {"app",
+        "-D", basedirProp, "-save", serviceName,
+        ExampleAppJson.resourceName(appDef),
+        "-appTypes", AppAdminClient.UNIT_TEST_TYPE};
     runCLI(args);
   }
 
   private void buildApp(String serviceName, String appDef, String lifetime,
       String queue) throws Throwable {
-    String[] args = {"app", "-D", basedirProp, "-save", serviceName,
-        ExampleAppJson.resourceName(appDef), "-updateLifetime", lifetime,
+    String[] args = {"app",
+        "-D", basedirProp, "-save", serviceName,
+        ExampleAppJson.resourceName(appDef),
+        "-appTypes", AppAdminClient.UNIT_TEST_TYPE,
+        "-updateLifetime", lifetime,
         "-changeQueue", queue};
     runCLI(args);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/438c1d33/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
index 6310178..dd7515f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
@@ -39,6 +39,9 @@ public abstract class AppAdminClient extends CompositeService {
       ".application.admin.client.class.";
   public static final String DEFAULT_TYPE = "yarn-service";
   public static final String DEFAULT_CLASS_NAME = "org.apache.hadoop.yarn" +
+      ".service.client.ApiServiceClient";
+  public static final String UNIT_TEST_TYPE = "unit-test";
+  public static final String UNIT_TEST_CLASS_NAME = "org.apache.hadoop.yarn" +
       ".service.client.ServiceClient";
 
   @Private
@@ -64,6 +67,9 @@ public abstract class AppAdminClient extends CompositeService {
     if (!clientClassMap.containsKey(DEFAULT_TYPE)) {
       clientClassMap.put(DEFAULT_TYPE, DEFAULT_CLASS_NAME);
     }
+    if (!clientClassMap.containsKey(UNIT_TEST_TYPE)) {
+      clientClassMap.put(UNIT_TEST_TYPE, UNIT_TEST_CLASS_NAME);
+    }
     if (!clientClassMap.containsKey(appType)) {
       throw new IllegalArgumentException("App admin client class name not " +
           "specified for type " + appType);


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