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 2018/05/18 21:30:52 UTC

[3/6] hadoop git commit: YARN-7530. Refactored YARN service API project location. Contributed by Chandni Singh

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a23ff8d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/SystemServiceManagerImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/SystemServiceManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/SystemServiceManagerImpl.java
new file mode 100644
index 0000000..f9cfa92
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/SystemServiceManagerImpl.java
@@ -0,0 +1,391 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.service.SystemServiceManager;
+import org.apache.hadoop.yarn.service.api.records.Service;
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
+import org.apache.hadoop.yarn.service.conf.YarnServiceConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.hadoop.yarn.service.utils.ServiceApiUtil.jsonSerDeser;
+
+/**
+ * SystemServiceManager implementation.
+ * Scan for configure system service path.
+ *
+ * The service path structure is as follows:
+ * SYSTEM_SERVICE_DIR_PATH
+ * |---- sync
+ * |     |--- user1
+ * |     |    |---- service1.yarnfile
+ * |     |    |---- service2.yarnfile
+ * |     |--- user2
+ * |     |    |---- service1.yarnfile
+ * |     |    ....
+ * |     |
+ * |---- async
+ * |     |--- user3
+ * |     |    |---- service1.yarnfile
+ * |     |    |---- service2.yarnfile
+ * |     |--- user4
+ * |     |    |---- service1.yarnfile
+ * |     |    ....
+ * |     |
+ *
+ * sync: These services are launched at the time of service start synchronously.
+ *       It is a blocking service start.
+ * async: These services are launched in separate thread without any delay after
+ *       service start. Non-blocking service start.
+ */
+public class SystemServiceManagerImpl extends AbstractService
+    implements SystemServiceManager {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SystemServiceManagerImpl.class);
+
+  private static final String YARN_FILE_SUFFIX = ".yarnfile";
+  private static final String SYNC = "sync";
+  private static final String ASYNC = "async";
+
+  private FileSystem fs;
+  private Path systemServiceDir;
+  private AtomicBoolean stopExecutors = new AtomicBoolean(false);
+  private Map<String, Set<Service>> syncUserServices = new HashMap<>();
+  private Map<String, Set<Service>> asyncUserServices = new HashMap<>();
+  private UserGroupInformation loginUGI;
+  private Thread serviceLaucher;
+
+  @VisibleForTesting
+  private int badFileNameExtensionSkipCounter;
+  @VisibleForTesting
+  private Map<String, Integer> ignoredUserServices =
+      new HashMap<>();
+  @VisibleForTesting
+  private int badDirSkipCounter;
+
+  public SystemServiceManagerImpl() {
+    super(SystemServiceManagerImpl.class.getName());
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    String dirPath =
+        conf.get(YarnServiceConf.YARN_SERVICES_SYSTEM_SERVICE_DIRECTORY);
+    if (dirPath != null) {
+      systemServiceDir = new Path(dirPath);
+      LOG.info("System Service Directory is configured to {}",
+          systemServiceDir);
+      fs = systemServiceDir.getFileSystem(conf);
+      this.loginUGI = UserGroupInformation.isSecurityEnabled() ?
+          UserGroupInformation.getLoginUser() :
+          UserGroupInformation.getCurrentUser();
+      LOG.info("UserGroupInformation initialized to {}", loginUGI);
+    }
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    scanForUserServices();
+    launchUserService(syncUserServices);
+    // Create a thread and submit services in background otherwise it
+    // block RM switch time.
+    serviceLaucher = new Thread(createRunnable());
+    serviceLaucher.setName("System service launcher");
+    serviceLaucher.start();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    LOG.info("Stopping {}", getName());
+    stopExecutors.set(true);
+
+    if (serviceLaucher != null) {
+      serviceLaucher.interrupt();
+      try {
+        serviceLaucher.join();
+      } catch (InterruptedException ie) {
+        LOG.warn("Interrupted Exception while stopping", ie);
+      }
+    }
+  }
+
+  private Runnable createRunnable() {
+    return new Runnable() {
+      @Override
+      public void run() {
+        launchUserService(asyncUserServices);
+      }
+    };
+  }
+
+  void launchUserService(Map<String, Set<Service>> userServices) {
+    for (Map.Entry<String, Set<Service>> entry : userServices.entrySet()) {
+      String user = entry.getKey();
+      Set<Service> services = entry.getValue();
+      if (services.isEmpty()) {
+        continue;
+      }
+      ServiceClient serviceClient = null;
+      try {
+        UserGroupInformation userUgi = getProxyUser(user);
+        serviceClient = createServiceClient(userUgi);
+        for (Service service : services) {
+          LOG.info("POST: createService = {} user = {}", service, userUgi);
+          try {
+            launchServices(userUgi, serviceClient, service);
+          } catch (IOException | UndeclaredThrowableException e) {
+            if (e.getCause() != null) {
+              LOG.warn(e.getCause().getMessage());
+            } else {
+              String message =
+                  "Failed to create service " + service.getName() + " : ";
+              LOG.error(message, e);
+            }
+          }
+        }
+      } catch (InterruptedException e) {
+        LOG.warn("System service launcher thread interrupted", e);
+        break;
+      } catch (Exception e) {
+        LOG.error("Error while submitting services for user " + user, e);
+      } finally {
+        if (serviceClient != null) {
+          try {
+            serviceClient.close();
+          } catch (IOException e) {
+            LOG.warn("Error while closing serviceClient for user {}", user);
+          }
+        }
+      }
+    }
+  }
+
+  private ServiceClient createServiceClient(UserGroupInformation userUgi)
+      throws IOException, InterruptedException {
+    ServiceClient serviceClient =
+        userUgi.doAs(new PrivilegedExceptionAction<ServiceClient>() {
+          @Override public ServiceClient run()
+              throws IOException, YarnException {
+            ServiceClient sc = getServiceClient();
+            sc.init(getConfig());
+            sc.start();
+            return sc;
+          }
+        });
+    return serviceClient;
+  }
+
+  private void launchServices(UserGroupInformation userUgi,
+      ServiceClient serviceClient, Service service)
+      throws IOException, InterruptedException {
+    if (service.getState() == ServiceState.STOPPED) {
+      userUgi.doAs(new PrivilegedExceptionAction<Void>() {
+        @Override public Void run() throws IOException, YarnException {
+          serviceClient.actionBuild(service);
+          return null;
+        }
+      });
+      LOG.info("Service {} version {} saved.", service.getName(),
+          service.getVersion());
+    } else {
+      ApplicationId applicationId =
+          userUgi.doAs(new PrivilegedExceptionAction<ApplicationId>() {
+            @Override public ApplicationId run()
+                throws IOException, YarnException {
+              ApplicationId applicationId = serviceClient.actionCreate(service);
+              return applicationId;
+            }
+          });
+      LOG.info("Service {} submitted with Application ID: {}",
+          service.getName(), applicationId);
+    }
+  }
+
+  ServiceClient getServiceClient() {
+    return new ServiceClient();
+  }
+
+  private UserGroupInformation getProxyUser(String user) {
+    UserGroupInformation ugi;
+    if (UserGroupInformation.isSecurityEnabled()) {
+      ugi = UserGroupInformation.createProxyUser(user, loginUGI);
+    } else {
+      ugi = UserGroupInformation.createRemoteUser(user);
+    }
+    return ugi;
+  }
+
+  // scan for both launch service types i.e sync and async
+  void scanForUserServices() throws IOException {
+    if (systemServiceDir == null) {
+      return;
+    }
+    try {
+      LOG.info("Scan for launch type on {}", systemServiceDir);
+      RemoteIterator<FileStatus> iterLaunchType = list(systemServiceDir);
+      while (iterLaunchType.hasNext()) {
+        FileStatus launchType = iterLaunchType.next();
+        if (!launchType.isDirectory()) {
+          LOG.debug("Scanner skips for unknown file {}", launchType.getPath());
+          continue;
+        }
+        if (launchType.getPath().getName().equals(SYNC)) {
+          scanForUserServiceDefinition(launchType.getPath(), syncUserServices);
+        } else if (launchType.getPath().getName().equals(ASYNC)) {
+          scanForUserServiceDefinition(launchType.getPath(), asyncUserServices);
+        } else {
+          badDirSkipCounter++;
+          LOG.debug("Scanner skips for unknown dir {}.", launchType.getPath());
+        }
+      }
+    } catch (FileNotFoundException e) {
+      LOG.warn("System service directory {} doesn't not exist.",
+          systemServiceDir);
+    }
+  }
+
+  // Files are under systemServiceDir/<users>. Scan for 2 levels
+  // 1st level for users
+  // 2nd level for service definitions under user
+  private void scanForUserServiceDefinition(Path userDirPath,
+      Map<String, Set<Service>> userServices) throws IOException {
+    LOG.info("Scan for users on {}", userDirPath);
+    RemoteIterator<FileStatus> iterUsers = list(userDirPath);
+    while (iterUsers.hasNext()) {
+      FileStatus userDir = iterUsers.next();
+      // if 1st level is not user directory then skip it.
+      if (!userDir.isDirectory()) {
+        LOG.info(
+            "Service definition {} doesn't belong to any user. Ignoring.. ",
+            userDir.getPath().getName());
+        continue;
+      }
+      String userName = userDir.getPath().getName();
+      LOG.info("Scanning service definitions for user {}.", userName);
+
+      //2nd level scan
+      RemoteIterator<FileStatus> iterServices = list(userDir.getPath());
+      while (iterServices.hasNext()) {
+        FileStatus serviceCache = iterServices.next();
+        String filename = serviceCache.getPath().getName();
+        if (!serviceCache.isFile()) {
+          LOG.info("Scanner skips for unknown dir {}", filename);
+          continue;
+        }
+        if (!filename.endsWith(YARN_FILE_SUFFIX)) {
+          LOG.info("Scanner skips for unknown file extension, filename = {}",
+              filename);
+          badFileNameExtensionSkipCounter++;
+          continue;
+        }
+        Service service = getServiceDefinition(serviceCache.getPath());
+        if (service != null) {
+          Set<Service> services = userServices.get(userName);
+          if (services == null) {
+            services = new HashSet<>();
+            userServices.put(userName, services);
+          }
+          if (!services.add(service)) {
+            int count = ignoredUserServices.containsKey(userName) ?
+                ignoredUserServices.get(userName) : 0;
+            ignoredUserServices.put(userName, count + 1);
+            LOG.warn(
+                "Ignoring service {} for the user {} as it is already present,"
+                    + " filename = {}", service.getName(), userName, filename);
+          } else {
+            LOG.info("Added service {} for the user {}, filename = {}",
+                service.getName(), userName, filename);
+          }
+        }
+      }
+    }
+  }
+
+  private Service getServiceDefinition(Path filePath) {
+    Service service = null;
+    try {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Loading service definition from FS: " + filePath);
+      }
+      service = jsonSerDeser.load(fs, filePath);
+    } catch (IOException e) {
+      LOG.info("Error while loading service definition from FS: {}", e);
+    }
+    return service;
+  }
+
+  private RemoteIterator<FileStatus> list(Path path) throws IOException {
+    return new StoppableRemoteIterator(fs.listStatusIterator(path));
+  }
+
+  @VisibleForTesting Map<String, Integer> getIgnoredUserServices() {
+    return ignoredUserServices;
+  }
+
+  private class StoppableRemoteIterator implements RemoteIterator<FileStatus> {
+    private final RemoteIterator<FileStatus> remote;
+
+    StoppableRemoteIterator(RemoteIterator<FileStatus> remote) {
+      this.remote = remote;
+    }
+
+    @Override public boolean hasNext() throws IOException {
+      return !stopExecutors.get() && remote.hasNext();
+    }
+
+    @Override public FileStatus next() throws IOException {
+      return remote.next();
+    }
+  }
+
+  @VisibleForTesting
+  Map<String, Set<Service>> getSyncUserServices() {
+    return syncUserServices;
+  }
+
+  @VisibleForTesting
+  int getBadFileNameExtensionSkipCounter() {
+    return badFileNameExtensionSkipCounter;
+  }
+
+  @VisibleForTesting
+  int getBadDirSkipCounter() {
+    return badDirSkipCounter;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a23ff8d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/package-info.java
new file mode 100644
index 0000000..cf5ce11
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/client/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.client contains classes
+ * for YARN Services Client API.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+package org.apache.hadoop.yarn.service.client;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a23ff8d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/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/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/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
new file mode 100644
index 0000000..46c9abe
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServer.java
@@ -0,0 +1,818 @@
+/*
+ * 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;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.VersionInfo;
+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.service.api.records.Component;
+import org.apache.hadoop.yarn.service.api.records.ComponentState;
+import org.apache.hadoop.yarn.service.api.records.Container;
+import org.apache.hadoop.yarn.service.api.records.ContainerState;
+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.conf.RestApiConstants;
+import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.servlet.http.HttpServletRequest;
+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.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+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.*;
+
+/**
+ * The rest API endpoints for users to manage services on YARN.
+ */
+@Singleton
+@Path(CONTEXT_ROOT)
+public class ApiServer {
+
+  public ApiServer() {
+    super();
+  }
+  
+  @Inject
+  public ApiServer(Configuration conf) {
+    super();
+  }
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ApiServer.class);
+  private static Configuration YARN_CONFIG = new YarnConfiguration();
+  private ServiceClient serviceClientUnitTest;
+  private boolean unitTest = false;
+
+  static {
+    init();
+  }
+
+  // initialize all the common resources - order is important
+  private static void init() {
+  }
+
+  @GET
+  @Path(VERSION)
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON + ";charset=utf-8" })
+  public Response getVersion() {
+    String version = VersionInfo.getBuildVersion();
+    LOG.info(version);
+    return Response.ok("{ \"hadoop_version\": \"" + version + "\"}").build();
+  }
+
+  @POST
+  @Path(SERVICE_ROOT_PATH)
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON + ";charset=utf-8" })
+  public Response createService(@Context HttpServletRequest request,
+      Service service) {
+    ServiceStatus serviceStatus = new ServiceStatus();
+    try {
+      UserGroupInformation ugi = getProxyUser(request);
+      LOG.info("POST: createService = {} user = {}", service, ugi);
+      if(service.getState()==ServiceState.STOPPED) {
+        ugi.doAs(new PrivilegedExceptionAction<Void>() {
+          @Override
+          public Void run() throws YarnException, IOException {
+            ServiceClient sc = getServiceClient();
+            sc.init(YARN_CONFIG);
+            sc.start();
+            sc.actionBuild(service);
+            sc.close();
+            return null;
+          }
+        });
+        serviceStatus.setDiagnostics("Service " + service.getName() +
+            " version " + service.getVersion() + " saved.");
+      } else {
+        ApplicationId applicationId = ugi
+            .doAs(new PrivilegedExceptionAction<ApplicationId>() {
+              @Override
+              public ApplicationId run() throws IOException, YarnException {
+                ServiceClient sc = getServiceClient();
+                sc.init(YARN_CONFIG);
+                sc.start();
+                ApplicationId applicationId = sc.actionCreate(service);
+                sc.close();
+                return applicationId;
+              }
+            });
+        serviceStatus.setDiagnostics("Application ID: " + applicationId);
+      }
+      serviceStatus.setState(ACCEPTED);
+      serviceStatus.setUri(
+          CONTEXT_ROOT + SERVICE_ROOT_PATH + "/" + service
+              .getName());
+      return formatResponse(Status.ACCEPTED, serviceStatus);
+    } catch (AccessControlException e) {
+      serviceStatus.setDiagnostics(e.getMessage());
+      return formatResponse(Status.FORBIDDEN, e.getCause().getMessage());
+    } catch (IllegalArgumentException e) {
+      return formatResponse(Status.BAD_REQUEST, e.getMessage());
+    } catch (IOException | InterruptedException e) {
+      String message = "Failed to create service " + service.getName()
+          + ": {}";
+      LOG.error(message, e);
+      return formatResponse(Status.INTERNAL_SERVER_ERROR, e.getMessage());
+    } catch (UndeclaredThrowableException e) {
+      String message = "Failed to create service " + service.getName()
+          + ": {}";
+      LOG.error(message, e);
+      if (e.getCause().getMessage().contains("already exists")) {
+        message = "Service name " + service.getName() + " is already taken.";
+      } else {
+        message = e.getCause().getMessage();
+      }
+      return formatResponse(Status.INTERNAL_SERVER_ERROR,
+          message);
+    }
+  }
+
+  @GET
+  @Path(SERVICE_PATH)
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON + ";charset=utf-8" })
+  public Response getService(@Context HttpServletRequest request,
+      @PathParam(SERVICE_NAME) String appName) {
+    ServiceStatus serviceStatus = new ServiceStatus();
+    try {
+      if (appName == null) {
+        throw new IllegalArgumentException("Service name cannot be null.");
+      }
+      UserGroupInformation ugi = getProxyUser(request);
+      LOG.info("GET: getService for appName = {} user = {}", appName, ugi);
+      Service app = getServiceFromClient(ugi, appName);
+      return Response.ok(app).build();
+    } catch (AccessControlException e) {
+      return formatResponse(Status.FORBIDDEN, e.getMessage());
+    } catch (IllegalArgumentException e) {
+      serviceStatus.setDiagnostics(e.getMessage());
+      serviceStatus.setCode(ERROR_CODE_APP_NAME_INVALID);
+      return Response.status(Status.NOT_FOUND).entity(serviceStatus)
+          .build();
+    } catch (FileNotFoundException e) {
+      serviceStatus.setDiagnostics("Service " + appName + " not found");
+      serviceStatus.setCode(ERROR_CODE_APP_NAME_INVALID);
+      return Response.status(Status.NOT_FOUND).entity(serviceStatus)
+          .build();
+    } catch (IOException | InterruptedException e) {
+      LOG.error("Get service failed: {}", e);
+      return formatResponse(Status.INTERNAL_SERVER_ERROR, e.getMessage());
+    } catch (UndeclaredThrowableException e) {
+      LOG.error("Get service failed: {}", e);
+      return formatResponse(Status.INTERNAL_SERVER_ERROR,
+          e.getCause().getMessage());
+    }
+  }
+
+  @DELETE
+  @Path(SERVICE_PATH)
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON + ";charset=utf-8" })
+  public Response deleteService(@Context HttpServletRequest request,
+      @PathParam(SERVICE_NAME) String appName) {
+    try {
+      if (appName == null) {
+        throw new IllegalArgumentException("Service name can not be null.");
+      }
+      UserGroupInformation ugi = getProxyUser(request);
+      LOG.info("DELETE: deleteService for appName = {} user = {}",
+          appName, ugi);
+      return stopService(appName, true, ugi);
+    } catch (AccessControlException e) {
+      return formatResponse(Status.FORBIDDEN, e.getMessage());
+    } catch (IllegalArgumentException e) {
+      return formatResponse(Status.BAD_REQUEST, e.getMessage());
+    } catch (UndeclaredThrowableException e) {
+      LOG.error("Fail to stop service: {}", e);
+      return formatResponse(Status.BAD_REQUEST,
+          e.getCause().getMessage());
+    } catch (YarnException | FileNotFoundException e) {
+      return formatResponse(Status.NOT_FOUND, e.getMessage());
+    } catch (Exception e) {
+      LOG.error("Fail to stop service: {}", e);
+      return formatResponse(Status.INTERNAL_SERVER_ERROR, e.getMessage());
+    }
+  }
+
+  private Response stopService(String appName, boolean destroy,
+      final UserGroupInformation ugi) throws Exception {
+    int result = ugi.doAs(new PrivilegedExceptionAction<Integer>() {
+      @Override
+      public Integer run() throws Exception {
+        int result = 0;
+        ServiceClient sc = getServiceClient();
+        sc.init(YARN_CONFIG);
+        sc.start();
+        Exception stopException = null;
+        try {
+          result = sc.actionStop(appName, destroy);
+          if (result == EXIT_SUCCESS) {
+            LOG.info("Successfully stopped service {}", appName);
+          }
+        } catch (Exception e) {
+          LOG.info("Got exception stopping service", e);
+          stopException = e;
+        }
+        if (destroy) {
+          result = sc.actionDestroy(appName);
+          if (result == EXIT_SUCCESS) {
+            LOG.info("Successfully deleted service {}", appName);
+          }
+        } else {
+          if (stopException != null) {
+            throw stopException;
+          }
+        }
+        sc.close();
+        return result;
+      }
+    });
+    ServiceStatus serviceStatus = new ServiceStatus();
+    if (destroy) {
+      if (result == EXIT_SUCCESS) {
+        serviceStatus.setDiagnostics("Successfully destroyed service " +
+            appName);
+      } else {
+        if (result == EXIT_NOT_FOUND) {
+          serviceStatus
+              .setDiagnostics("Service " + appName + " doesn't exist");
+          return formatResponse(Status.BAD_REQUEST, serviceStatus);
+        } else {
+          serviceStatus
+              .setDiagnostics("Service " + appName + " error cleaning up " +
+                  "registry");
+          return formatResponse(Status.INTERNAL_SERVER_ERROR, serviceStatus);
+        }
+      }
+    } else {
+      if (result == EXIT_COMMAND_ARGUMENT_ERROR) {
+        serviceStatus
+            .setDiagnostics("Service " + appName + " is already stopped");
+        return formatResponse(Status.BAD_REQUEST, serviceStatus);
+      } else {
+        serviceStatus.setDiagnostics("Successfully stopped service " + appName);
+      }
+    }
+    return formatResponse(Status.OK, serviceStatus);
+  }
+
+  @PUT
+  @Path(COMPONENTS_PATH)
+  @Consumes({MediaType.APPLICATION_JSON})
+  @Produces({RestApiConstants.MEDIA_TYPE_JSON_UTF8, MediaType.TEXT_PLAIN})
+  public Response updateComponents(@Context HttpServletRequest request,
+      @PathParam(SERVICE_NAME) String serviceName,
+      List<Component> requestComponents) {
+
+    try {
+      if (requestComponents == null || requestComponents.isEmpty()) {
+        throw new YarnException("No components provided.");
+      }
+      UserGroupInformation ugi = getProxyUser(request);
+      Set<String> compNamesToUpgrade = new HashSet<>();
+      requestComponents.forEach(reqComp -> {
+        if (reqComp.getState() != null &&
+            reqComp.getState().equals(ComponentState.UPGRADING)) {
+          compNamesToUpgrade.add(reqComp.getName());
+        }
+      });
+      LOG.info("PUT: upgrade components {} for service {} " +
+          "user = {}", compNamesToUpgrade, serviceName, ugi);
+      return processComponentsUpgrade(ugi, serviceName, compNamesToUpgrade);
+    } catch (AccessControlException e) {
+      return formatResponse(Response.Status.FORBIDDEN, e.getMessage());
+    } catch (YarnException e) {
+      return formatResponse(Response.Status.BAD_REQUEST, e.getMessage());
+    } catch (IOException | InterruptedException e) {
+      return formatResponse(Response.Status.INTERNAL_SERVER_ERROR,
+          e.getMessage());
+    } catch (UndeclaredThrowableException e) {
+      return formatResponse(Response.Status.INTERNAL_SERVER_ERROR,
+          e.getCause().getMessage());
+    }
+  }
+
+  @PUT
+  @Path(COMPONENT_PATH)
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON + ";charset=utf-8",
+              MediaType.TEXT_PLAIN  })
+  public Response updateComponent(@Context HttpServletRequest request,
+      @PathParam(SERVICE_NAME) String appName,
+      @PathParam(COMPONENT_NAME) String componentName, Component component) {
+
+    try {
+      if (component == null) {
+        throw new YarnException("No component data provided");
+      }
+      if (component.getName() != null
+          && !component.getName().equals(componentName)) {
+        String msg = "Component name in the request object ("
+            + component.getName() + ") does not match that in the URI path ("
+            + componentName + ")";
+        throw new YarnException(msg);
+      }
+      UserGroupInformation ugi = getProxyUser(request);
+      if (component.getState() != null &&
+          component.getState().equals(ComponentState.UPGRADING)) {
+        LOG.info("PUT: upgrade component {} for service {} " +
+            "user = {}", component.getName(), appName, ugi);
+        return processComponentsUpgrade(ugi, appName,
+            Sets.newHashSet(componentName));
+      }
+
+      if (component.getNumberOfContainers() == null) {
+        throw new YarnException("No container count provided");
+      }
+      if (component.getNumberOfContainers() < 0) {
+        String message = "Invalid number of containers specified "
+            + component.getNumberOfContainers();
+        throw new YarnException(message);
+      }
+      Map<String, Long> original = ugi
+          .doAs(new PrivilegedExceptionAction<Map<String, Long>>() {
+            @Override
+            public Map<String, Long> run() throws YarnException, IOException {
+              ServiceClient sc = new ServiceClient();
+              sc.init(YARN_CONFIG);
+              sc.start();
+              Map<String, Long> original = sc.flexByRestService(appName,
+                  Collections.singletonMap(componentName,
+                      component.getNumberOfContainers()));
+              sc.close();
+              return original;
+            }
+          });
+      ServiceStatus status = new ServiceStatus();
+      status.setDiagnostics(
+          "Updating component (" + componentName + ") size from " + original
+              .get(componentName) + " to " + component.getNumberOfContainers());
+      return formatResponse(Status.OK, status);
+    } catch (AccessControlException e) {
+      return formatResponse(Status.FORBIDDEN, e.getMessage());
+    } catch (YarnException e) {
+      return formatResponse(Status.BAD_REQUEST, e.getMessage());
+    } catch (IOException | InterruptedException e) {
+      return formatResponse(Status.INTERNAL_SERVER_ERROR,
+          e.getMessage());
+    } catch (UndeclaredThrowableException e) {
+      return formatResponse(Status.INTERNAL_SERVER_ERROR,
+          e.getCause().getMessage());
+    }
+  }
+
+  @PUT
+  @Path(SERVICE_PATH)
+  @Consumes({ MediaType.APPLICATION_JSON })
+  @Produces({ MediaType.APPLICATION_JSON + ";charset=utf-8" })
+  public Response updateService(@Context HttpServletRequest request,
+      @PathParam(SERVICE_NAME) String appName,
+      Service updateServiceData) {
+    try {
+      UserGroupInformation ugi = getProxyUser(request);
+      LOG.info("PUT: updateService for app = {} with data = {} user = {}",
+          appName, updateServiceData, ugi);
+      // Ignore the app name provided in updateServiceData and always use
+      // appName path param
+      updateServiceData.setName(appName);
+
+      if (updateServiceData.getState() != null
+          && updateServiceData.getState() == ServiceState.FLEX) {
+        return flexService(updateServiceData, ugi);
+      }
+      // 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.
+      if (updateServiceData.getState() != null
+          && updateServiceData.getState() == ServiceState.STOPPED) {
+        return stopService(appName, false, ugi);
+      }
+
+      // If a START is requested
+      if (updateServiceData.getState() != null
+          && updateServiceData.getState() == ServiceState.STARTED) {
+        return startService(appName, ugi);
+      }
+
+      // If an UPGRADE is requested
+      if (updateServiceData.getState() != null && (
+          updateServiceData.getState() == ServiceState.UPGRADING ||
+              updateServiceData.getState() ==
+                  ServiceState.UPGRADING_AUTO_FINALIZE)) {
+        return upgradeService(updateServiceData, ugi);
+      }
+
+      // If new lifetime value specified then update it
+      if (updateServiceData.getLifetime() != null
+          && updateServiceData.getLifetime() > 0) {
+        return updateLifetime(appName, updateServiceData, ugi);
+      }
+    } catch (UndeclaredThrowableException e) {
+      return formatResponse(Status.BAD_REQUEST,
+          e.getCause().getMessage());
+    } catch (AccessControlException e) {
+      return formatResponse(Status.FORBIDDEN, e.getMessage());
+    } catch (FileNotFoundException e) {
+      String message = "Application is not found app: " + appName;
+      LOG.error(message, e);
+      return formatResponse(Status.NOT_FOUND, e.getMessage());
+    } catch (YarnException e) {
+      String message = "Service is not found in hdfs: " + appName;
+      LOG.error(message, e);
+      return formatResponse(Status.NOT_FOUND, e.getMessage());
+    } catch (Exception e) {
+      String message = "Error while performing operation for app: " + appName;
+      LOG.error(message, e);
+      return formatResponse(Status.INTERNAL_SERVER_ERROR, e.getMessage());
+    }
+
+    // If nothing happens consider it a no-op
+    return Response.status(Status.NO_CONTENT).build();
+  }
+
+  @PUT
+  @Path(COMP_INSTANCE_LONG_PATH)
+  @Consumes({MediaType.APPLICATION_JSON})
+  @Produces({RestApiConstants.MEDIA_TYPE_JSON_UTF8, MediaType.TEXT_PLAIN})
+  public Response updateComponentInstance(@Context HttpServletRequest request,
+      @PathParam(SERVICE_NAME) String serviceName,
+      @PathParam(COMPONENT_NAME) String componentName,
+      @PathParam(COMP_INSTANCE_NAME) String compInstanceName,
+      Container reqContainer) {
+
+    try {
+      UserGroupInformation ugi = getProxyUser(request);
+      LOG.info("PUT: update component instance {} for component = {}" +
+              " service = {} user = {}", compInstanceName, componentName,
+          serviceName, ugi);
+      if (reqContainer == null) {
+        throw new YarnException("No container data provided.");
+      }
+      Service service = getServiceFromClient(ugi, serviceName);
+      Component component = service.getComponent(componentName);
+      if (component == null) {
+        throw new YarnException(String.format(
+            "The component name in the URI path (%s) is invalid.",
+            componentName));
+      }
+
+      Container liveContainer = component.getComponentInstance(
+          compInstanceName);
+      if (liveContainer == null) {
+        throw new YarnException(String.format(
+            "The component (%s) does not have a component instance (%s).",
+            componentName, compInstanceName));
+      }
+
+      if (reqContainer.getState() != null
+          && reqContainer.getState().equals(ContainerState.UPGRADING)) {
+        return processContainersUpgrade(ugi, service,
+            Lists.newArrayList(liveContainer));
+      }
+    } catch (AccessControlException e) {
+      return formatResponse(Response.Status.FORBIDDEN, e.getMessage());
+    } catch (YarnException e) {
+      return formatResponse(Response.Status.BAD_REQUEST, e.getMessage());
+    } catch (IOException | InterruptedException e) {
+      return formatResponse(Response.Status.INTERNAL_SERVER_ERROR,
+          e.getMessage());
+    } catch (UndeclaredThrowableException e) {
+      return formatResponse(Response.Status.INTERNAL_SERVER_ERROR,
+          e.getCause().getMessage());
+    }
+    return Response.status(Status.NO_CONTENT).build();
+  }
+
+  @PUT
+  @Path(COMP_INSTANCES_PATH)
+  @Consumes({MediaType.APPLICATION_JSON})
+  @Produces({RestApiConstants.MEDIA_TYPE_JSON_UTF8, MediaType.TEXT_PLAIN})
+  public Response updateComponentInstances(@Context HttpServletRequest request,
+      @PathParam(SERVICE_NAME) String serviceName,
+      List<Container> requestContainers) {
+
+    try {
+      if (requestContainers == null || requestContainers.isEmpty()) {
+        throw new YarnException("No containers provided.");
+      }
+      UserGroupInformation ugi = getProxyUser(request);
+      List<String> toUpgrade = new ArrayList<>();
+      for (Container reqContainer : requestContainers) {
+        if (reqContainer.getState() != null &&
+            reqContainer.getState().equals(ContainerState.UPGRADING)) {
+          toUpgrade.add(reqContainer.getComponentInstanceName());
+        }
+      }
+
+      if (!toUpgrade.isEmpty()) {
+        Service service = getServiceFromClient(ugi, serviceName);
+        LOG.info("PUT: upgrade component instances {} for service = {} " +
+            "user = {}", toUpgrade, serviceName, ugi);
+        List<Container> liveContainers = ServiceApiUtil
+            .getLiveContainers(service, toUpgrade);
+
+        return processContainersUpgrade(ugi, service, liveContainers);
+      }
+    } catch (AccessControlException e) {
+      return formatResponse(Response.Status.FORBIDDEN, e.getMessage());
+    } catch (YarnException e) {
+      return formatResponse(Response.Status.BAD_REQUEST, e.getMessage());
+    } catch (IOException | InterruptedException e) {
+      return formatResponse(Response.Status.INTERNAL_SERVER_ERROR,
+          e.getMessage());
+    } catch (UndeclaredThrowableException e) {
+      return formatResponse(Response.Status.INTERNAL_SERVER_ERROR,
+          e.getCause().getMessage());
+    }
+    return Response.status(Status.NO_CONTENT).build();
+  }
+
+  private Response flexService(Service service, UserGroupInformation ugi)
+      throws IOException, InterruptedException {
+    String appName = service.getName();
+    Response response = Response.status(Status.BAD_REQUEST).build();
+    Map<String, String> componentCountStrings = new HashMap<String, String>();
+    for (Component c : service.getComponents()) {
+      componentCountStrings.put(c.getName(),
+          c.getNumberOfContainers().toString());
+    }
+    Integer result = ugi.doAs(new PrivilegedExceptionAction<Integer>() {
+
+      @Override
+      public Integer run() throws YarnException, IOException {
+        int result = 0;
+        ServiceClient sc = new ServiceClient();
+        sc.init(YARN_CONFIG);
+        sc.start();
+        result = sc
+            .actionFlex(appName, componentCountStrings);
+        sc.close();
+        return Integer.valueOf(result);
+      }
+    });
+    if (result == EXIT_SUCCESS) {
+      String message = "Service " + appName + " is successfully flexed.";
+      LOG.info(message);
+      ServiceStatus status = new ServiceStatus();
+      status.setDiagnostics(message);
+      status.setState(ServiceState.ACCEPTED);
+      response = formatResponse(Status.ACCEPTED, status);
+    }
+    return response;
+  }
+
+  private Response updateLifetime(String appName, Service updateAppData,
+      final UserGroupInformation ugi) throws IOException,
+      InterruptedException {
+    String newLifeTime = ugi.doAs(new PrivilegedExceptionAction<String>() {
+      @Override
+      public String run() throws YarnException, IOException {
+        ServiceClient sc = getServiceClient();
+        sc.init(YARN_CONFIG);
+        sc.start();
+        String newLifeTime = sc.updateLifetime(appName,
+            updateAppData.getLifetime());
+        sc.close();
+        return newLifeTime;
+      }
+    });
+    ServiceStatus status = new ServiceStatus();
+    status.setDiagnostics(
+        "Service (" + appName + ")'s lifeTime is updated to " + newLifeTime
+            + ", " + updateAppData.getLifetime() + " seconds remaining");
+    return formatResponse(Status.OK, status);
+  }
+
+  private Response startService(String appName,
+      final UserGroupInformation ugi) throws IOException,
+      InterruptedException {
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws YarnException, IOException {
+        ServiceClient sc = getServiceClient();
+        sc.init(YARN_CONFIG);
+        sc.start();
+        sc.actionStart(appName);
+        sc.close();
+        return null;
+      }
+    });
+    LOG.info("Successfully started service " + appName);
+    ServiceStatus status = new ServiceStatus();
+    status.setDiagnostics("Service " + appName + " is successfully started.");
+    status.setState(ServiceState.ACCEPTED);
+    return formatResponse(Status.OK, status);
+  }
+
+  private Response upgradeService(Service service,
+      final UserGroupInformation ugi) throws IOException, InterruptedException {
+    ServiceStatus status = new ServiceStatus();
+    ugi.doAs((PrivilegedExceptionAction<Void>) () -> {
+      ServiceClient sc = getServiceClient();
+      sc.init(YARN_CONFIG);
+      sc.start();
+      sc.initiateUpgrade(service);
+      sc.close();
+      return null;
+    });
+    LOG.info("Service {} version {} upgrade initialized", service.getName(),
+        service.getVersion());
+    status.setDiagnostics("Service " + service.getName() +
+        " version " + service.getVersion() + " saved.");
+    status.setState(ServiceState.ACCEPTED);
+    return formatResponse(Status.ACCEPTED, status);
+  }
+
+  private Response processComponentsUpgrade(UserGroupInformation ugi,
+      String serviceName, Set<String> compNames) throws YarnException,
+      IOException, InterruptedException {
+    Service service = getServiceFromClient(ugi, serviceName);
+    if (service.getState() != ServiceState.UPGRADING) {
+      throw new YarnException(
+          String.format("The upgrade of service %s has not been initiated.",
+              service.getName()));
+    }
+    List<Container> containersToUpgrade = ServiceApiUtil
+        .validateAndResolveCompsUpgrade(service, compNames);
+    Integer result = invokeContainersUpgrade(ugi, service, containersToUpgrade);
+    if (result == EXIT_SUCCESS) {
+      ServiceStatus status = new ServiceStatus();
+      status.setDiagnostics(
+          "Upgrading components " + Joiner.on(',').join(compNames) + ".");
+      return formatResponse(Response.Status.ACCEPTED, status);
+    }
+    // If result is not a success, consider it a no-op
+    return Response.status(Response.Status.NO_CONTENT).build();
+  }
+
+  private Response processContainersUpgrade(UserGroupInformation ugi,
+      Service service, List<Container> containers) throws YarnException,
+      IOException, InterruptedException {
+
+    if (service.getState() != ServiceState.UPGRADING) {
+      throw new YarnException(
+          String.format("The upgrade of service %s has not been initiated.",
+              service.getName()));
+    }
+    ServiceApiUtil.validateInstancesUpgrade(containers);
+    Integer result = invokeContainersUpgrade(ugi, service, containers);
+    if (result == EXIT_SUCCESS) {
+      ServiceStatus status = new ServiceStatus();
+      status.setDiagnostics(
+          "Upgrading component instances " + containers.stream()
+              .map(Container::getId).collect(Collectors.joining(",")) + ".");
+      return formatResponse(Response.Status.ACCEPTED, status);
+    }
+    // If result is not a success, consider it a no-op
+    return Response.status(Response.Status.NO_CONTENT).build();
+  }
+
+  private int invokeContainersUpgrade(UserGroupInformation ugi,
+      Service service, List<Container> containers) throws IOException,
+      InterruptedException {
+    return ugi.doAs((PrivilegedExceptionAction<Integer>) () -> {
+      int result1;
+      ServiceClient sc = getServiceClient();
+      sc.init(YARN_CONFIG);
+      sc.start();
+      result1 = sc.actionUpgrade(service, containers);
+      sc.close();
+      return result1;
+    });
+  }
+
+  private Service getServiceFromClient(UserGroupInformation ugi,
+      String serviceName) throws IOException, InterruptedException {
+
+    return ugi.doAs((PrivilegedExceptionAction<Service>) () -> {
+      ServiceClient sc = getServiceClient();
+      sc.init(YARN_CONFIG);
+      sc.start();
+      Service app1 = sc.getStatus(serviceName);
+      sc.close();
+      return app1;
+    });
+  }
+
+  /**
+   * Used by negative test case.
+   *
+   * @param mockServerClient - A mocked version of ServiceClient
+   */
+  public void setServiceClient(ServiceClient mockServerClient) {
+    serviceClientUnitTest = mockServerClient;
+    unitTest = true;
+  }
+
+  private ServiceClient getServiceClient() {
+    if (unitTest) {
+      return serviceClientUnitTest;
+    } else {
+      return new ServiceClient();
+    }
+  }
+
+  /**
+   * Configure impersonation callback.
+   *
+   * @param request - web request
+   * @return - configured UGI class for proxy callback
+   * @throws IOException - if user is not login.
+   */
+  private UserGroupInformation getProxyUser(HttpServletRequest request)
+      throws AccessControlException {
+    UserGroupInformation proxyUser;
+    UserGroupInformation ugi;
+    String remoteUser = request.getRemoteUser();
+    try {
+      if (UserGroupInformation.isSecurityEnabled()) {
+        proxyUser = UserGroupInformation.getLoginUser();
+        ugi = UserGroupInformation.createProxyUser(remoteUser, proxyUser);
+      } else {
+        ugi = UserGroupInformation.createRemoteUser(remoteUser);
+      }
+      return ugi;
+    } catch (IOException e) {
+      throw new AccessControlException(e.getCause());
+    }
+  }
+
+  /**
+   * Format HTTP response.
+   *
+   * @param status - HTTP Code
+   * @param message - Diagnostic message
+   * @return - HTTP response
+   */
+  private Response formatResponse(Status status, String message) {
+    ServiceStatus entity = new ServiceStatus();
+    entity.setDiagnostics(message);
+    return formatResponse(status, entity);
+  }
+
+  /**
+   * Format HTTP response.
+   *
+   * @param status - HTTP Code
+   * @param entity - ServiceStatus object
+   * @return - HTTP response
+   */
+  private Response formatResponse(Status status, ServiceStatus entity) {
+    return Response.status(status).entity(entity).build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a23ff8d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServerWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServerWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServerWebApp.java
new file mode 100644
index 0000000..f4acd94
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/java/org/apache/hadoop/yarn/service/webapp/ApiServerWebApp.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.service.webapp;
+
+import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AuthenticationFilterInitializer;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
+import org.eclipse.jetty.webapp.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.RM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY;
+import static org.apache.hadoop.yarn.conf.YarnConfiguration.RM_WEBAPP_SPNEGO_USER_NAME_KEY;
+import static org.apache.hadoop.yarn.service.conf.YarnServiceConf.*;
+
+/**
+ * This class launches the web service using Hadoop HttpServer2 (which uses
+ * an embedded Jetty container). This is the entry point to your service.
+ * The Java command used to launch this app should call the main method.
+ */
+public class ApiServerWebApp extends AbstractService {
+  private static final Logger logger = LoggerFactory
+      .getLogger(ApiServerWebApp.class);
+  private static final String SEP = ";";
+
+  // REST API server for YARN native services
+  private HttpServer2 apiServer;
+  private InetSocketAddress bindAddress;
+
+  public static void main(String[] args) throws IOException {
+    ApiServerWebApp apiWebApp = new ApiServerWebApp();
+    try {
+      apiWebApp.init(new YarnConfiguration());
+      apiWebApp.serviceStart();
+    } catch (Exception e) {
+      logger.error("Got exception starting", e);
+      apiWebApp.close();
+    }
+  }
+
+  public ApiServerWebApp() {
+    super(ApiServerWebApp.class.getName());
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    bindAddress = getConfig().getSocketAddr(API_SERVER_ADDRESS,
+        DEFAULT_API_SERVER_ADDRESS, DEFAULT_API_SERVER_PORT);
+    logger.info("YARN API server running on " + bindAddress);
+    if (UserGroupInformation.isSecurityEnabled()) {
+      doSecureLogin(getConfig());
+    }
+    startWebApp();
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    if (apiServer != null) {
+      apiServer.stop();
+    }
+    super.serviceStop();
+  }
+
+  private void doSecureLogin(org.apache.hadoop.conf.Configuration conf)
+      throws IOException {
+    SecurityUtil.login(conf, YarnConfiguration.RM_KEYTAB,
+        YarnConfiguration.RM_PRINCIPAL, bindAddress.getHostName());
+    addFilters(conf);
+  }
+
+  private void addFilters(org.apache.hadoop.conf.Configuration conf) {
+    // Always load pseudo authentication filter to parse "user.name" in an URL
+    // to identify a HTTP request's user.
+    boolean hasHadoopAuthFilterInitializer = false;
+    String filterInitializerConfKey = "hadoop.http.filter.initializers";
+    Class<?>[] initializersClasses =
+        conf.getClasses(filterInitializerConfKey);
+    List<String> targets = new ArrayList<String>();
+    if (initializersClasses != null) {
+      for (Class<?> initializer : initializersClasses) {
+        if (initializer.getName().equals(
+            AuthenticationFilterInitializer.class.getName())) {
+          hasHadoopAuthFilterInitializer = true;
+          break;
+        }
+        targets.add(initializer.getName());
+      }
+    }
+    if (!hasHadoopAuthFilterInitializer) {
+      targets.add(AuthenticationFilterInitializer.class.getName());
+      conf.set(filterInitializerConfKey, StringUtils.join(",", targets));
+    }
+  }
+
+  private void startWebApp() throws IOException {
+    URI uri = URI.create("http://" + NetUtils.getHostPortString(bindAddress));
+
+    apiServer = new HttpServer2.Builder()
+        .setName("api-server")
+        .setConf(getConfig())
+        .setSecurityEnabled(UserGroupInformation.isSecurityEnabled())
+        .setUsernameConfKey(RM_WEBAPP_SPNEGO_USER_NAME_KEY)
+        .setKeytabConfKey(RM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY)
+        .addEndpoint(uri).build();
+
+    String apiPackages =
+        ApiServer.class.getPackage().getName() + SEP
+            + GenericExceptionHandler.class.getPackage().getName() + SEP
+            + YarnJacksonJaxbJsonProvider.class.getPackage().getName();
+    apiServer.addJerseyResourcePackage(apiPackages, "/*");
+
+    try {
+      logger.info("Service starting up. Logging start...");
+      apiServer.start();
+      logger.info("Server status = {}", apiServer.toString());
+      for (Configuration conf : apiServer.getWebAppContext()
+          .getConfigurations()) {
+        logger.info("Configurations = {}", conf);
+      }
+      logger.info("Context Path = {}", Collections.singletonList(
+          apiServer.getWebAppContext().getContextPath()));
+      logger.info("ResourceBase = {}", Collections.singletonList(
+          apiServer.getWebAppContext().getResourceBase()));
+      logger.info("War = {}", Collections
+          .singletonList(apiServer.getWebAppContext().getWar()));
+    } catch (Exception ex) {
+      logger.error("Hadoop HttpServer2 App **failed**", ex);
+      throw ex;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a23ff8d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/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/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/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/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/a23ff8d8/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/resources/definition/YARN-Services-Examples.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/resources/definition/YARN-Services-Examples.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/resources/definition/YARN-Services-Examples.md
new file mode 100644
index 0000000..b7ad6c9
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/resources/definition/YARN-Services-Examples.md
@@ -0,0 +1,444 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+## Examples
+
+### Create a simple single-component service with most attribute values as defaults
+POST URL - http://localhost:8088/app/v1/services
+
+##### POST Request JSON
+```json
+{
+  "name": "hello-world",
+  "version": "1.0.0",
+  "description": "hello world example",
+  "components" :
+    [
+      {
+        "name": "hello",
+        "number_of_containers": 2,
+        "artifact": {
+          "id": "nginx:latest",
+          "type": "DOCKER"
+        },
+        "launch_command": "./start_nginx.sh",
+        "resource": {
+          "cpus": 1,
+          "memory": "256"
+        }
+      }
+    ]
+}
+```
+
+##### GET Response JSON
+GET URL - http://localhost:8088/app/v1/services/hello-world
+
+Note, lifetime value of -1 means unlimited lifetime.
+
+```json
+{
+    "name": "hello-world",
+    "version": "1.0.0",
+    "description": "hello world example",
+    "id": "application_1503963985568_0002",
+    "lifetime": -1,
+    "state": "STABLE",
+    "components": [
+        {
+            "name": "hello",
+            "state": "STABLE",
+            "resource": {
+                "cpus": 1,
+                "memory": "256"
+            },
+            "configuration": {
+                "properties": {},
+                "env": {},
+                "files": []
+            },
+            "quicklinks": [],
+            "containers": [
+                {
+                    "id": "container_e03_1503963985568_0002_01_000002",
+                    "ip": "10.22.8.143",
+                    "hostname": "ctr-e03-1503963985568-0002-01-000002.example.site",
+                    "state": "READY",
+                    "launch_time": 1504051512412,
+                    "bare_host": "host100.cloud.com",
+                    "component_instance_name": "hello-0"
+                },
+                {
+                    "id": "container_e03_1503963985568_0002_01_000003",
+                    "ip": "10.22.8.144",
+                    "hostname": "ctr-e03-1503963985568-0002-01-000003.example.site",
+                    "state": "READY",
+                    "launch_time": 1504051536450,
+                    "bare_host": "host100.cloud.com",
+                    "component_instance_name": "hello-1"
+                }
+            ],
+            "launch_command": "./start_nginx.sh",
+            "number_of_containers": 1,
+            "run_privileged_container": false
+        }
+    ],
+    "configuration": {
+        "properties": {},
+        "env": {},
+        "files": []
+    },
+    "quicklinks": {}
+}
+
+```
+### Update to modify the lifetime of a service
+PUT URL - http://localhost:8088/app/v1/services/hello-world
+
+##### PUT Request JSON
+
+Note, irrespective of what the current lifetime value is, this update request will set the lifetime of the service to be 3600 seconds (1 hour) from the time the request is submitted. Hence, if a a service has remaining lifetime of 5 mins (say) and would like to extend it to an hour OR if an application has remaining lifetime of 5 hours (say) and would like to reduce it down to an hour, then for both scenarios you need to submit the same request below.
+
+```json
+{
+  "lifetime": 3600
+}
+```
+### Stop a service
+PUT URL - http://localhost:8088/app/v1/services/hello-world
+
+##### PUT Request JSON
+```json
+{
+  "state": "STOPPED"
+}
+```
+
+### Start a service
+PUT URL - http://localhost:8088/app/v1/services/hello-world
+
+##### PUT Request JSON
+```json
+{
+  "state": "STARTED"
+}
+```
+
+### Update to flex up/down the number of containers (instances) of a component of a service
+PUT URL - http://localhost:8088/app/v1/services/hello-world/components/hello
+
+##### PUT Request JSON
+```json
+{
+  "number_of_containers": 3
+}
+```
+
+Alternatively, you can specify the entire "components" section instead.
+
+PUT URL - http://localhost:8088/app/v1/services/hello-world
+##### PUT Request JSON
+```json
+{
+  "state": "FLEX",
+  "components" :
+    [
+      {
+        "name": "hello",
+        "number_of_containers": 3
+      }
+    ]
+}
+```
+
+### Destroy a service
+DELETE URL - http://localhost:8088/app/v1/services/hello-world
+
+***
+
+### Create a complicated service  - HBase
+POST URL - http://localhost:8088:/app/v1/services/hbase-app-1
+
+##### POST Request JSON
+
+```json
+{
+  "name": "hbase-app-1",
+  "version": "1.0.0",
+  "description": "hbase service",
+  "lifetime": "3600",
+  "components": [
+    {
+      "name": "hbasemaster",
+      "number_of_containers": 1,
+      "artifact": {
+        "id": "hbase:latest",
+        "type": "DOCKER"
+      },
+      "launch_command": "/usr/hdp/current/hbase-master/bin/hbase master start",
+      "resource": {
+        "cpus": 1,
+        "memory": "2048"
+      },
+      "configuration": {
+        "env": {
+          "HBASE_LOG_DIR": "<LOG_DIR>"
+        },
+        "files": [
+          {
+            "type": "XML",
+            "dest_file": "/etc/hadoop/conf/core-site.xml",
+            "properties": {
+              "fs.defaultFS": "${CLUSTER_FS_URI}"
+            }
+          },
+          {
+            "type": "XML",
+            "dest_file": "/etc/hbase/conf/hbase-site.xml",
+            "properties": {
+              "hbase.cluster.distributed": "true",
+              "hbase.zookeeper.quorum": "${CLUSTER_ZK_QUORUM}",
+              "hbase.rootdir": "${SERVICE_HDFS_DIR}/hbase",
+              "zookeeper.znode.parent": "${SERVICE_ZK_PATH}",
+              "hbase.master.hostname": "hbasemaster.${SERVICE_NAME}.${USER}.${DOMAIN}",
+              "hbase.master.info.port": "16010"
+            }
+          }
+        ]
+      }
+    },
+    {
+      "name": "regionserver",
+      "number_of_containers": 3,
+      "artifact": {
+        "id": "hbase:latest",
+        "type": "DOCKER"
+      },
+      "launch_command": "/usr/hdp/current/hbase-regionserver/bin/hbase regionserver start",
+      "resource": {
+        "cpus": 1,
+        "memory": "2048"
+      },
+      "configuration": {
+        "env": {
+          "HBASE_LOG_DIR": "<LOG_DIR>"
+        },
+        "files": [
+          {
+            "type": "XML",
+            "dest_file": "/etc/hadoop/conf/core-site.xml",
+            "properties": {
+              "fs.defaultFS": "${CLUSTER_FS_URI}"
+            }
+          },
+          {
+            "type": "XML",
+            "dest_file": "/etc/hbase/conf/hbase-site.xml",
+            "properties": {
+              "hbase.cluster.distributed": "true",
+              "hbase.zookeeper.quorum": "${CLUSTER_ZK_QUORUM}",
+              "hbase.rootdir": "${SERVICE_HDFS_DIR}/hbase",
+              "zookeeper.znode.parent": "${SERVICE_ZK_PATH}",
+              "hbase.master.hostname": "hbasemaster.${SERVICE_NAME}.${USER}.${DOMAIN}",
+              "hbase.master.info.port": "16010",
+              "hbase.regionserver.hostname": "${COMPONENT_INSTANCE_NAME}.${SERVICE_NAME}.${USER}.${DOMAIN}"
+            }
+          }
+        ]
+      }
+    }
+  ],
+  "quicklinks": {
+    "HBase Master Status UI": "http://hbasemaster0.${SERVICE_NAME}.${USER}.${DOMAIN}:16010/master-status",
+    "Proxied HBase Master Status UI": "http://app-proxy/${DOMAIN}/${USER}/${SERVICE_NAME}/hbasemaster/16010/"
+  }
+}
+```
+
+### Create a service requesting GPUs in addition to CPUs and RAM
+POST URL - http://localhost:8088/app/v1/services
+
+##### POST Request JSON
+```json
+{
+  "name": "hello-world",
+  "version": "1.0.0",
+  "description": "hello world example with GPUs",
+  "components" :
+    [
+      {
+        "name": "hello",
+        "number_of_containers": 2,
+        "artifact": {
+          "id": "nginx:latest",
+          "type": "DOCKER"
+        },
+        "launch_command": "./start_nginx.sh",
+        "resource": {
+          "cpus": 1,
+          "memory": "256",
+          "additional" : {
+            "yarn.io/gpu" : {
+              "value" : 4,
+              "unit" : ""
+            }
+          }
+        }
+      }
+    ]
+}
+```
+
+### Create a service with a component requesting anti-affinity placement policy
+POST URL - http://localhost:8088/app/v1/services
+
+##### POST Request JSON
+```json
+{
+  "name": "hello-world",
+  "version": "1.0.0",
+  "description": "hello world example with anti-affinity",
+  "components" :
+    [
+      {
+        "name": "hello",
+        "number_of_containers": 3,
+        "artifact": {
+          "id": "nginx:latest",
+          "type": "DOCKER"
+        },
+        "launch_command": "./start_nginx.sh",
+        "resource": {
+          "cpus": 1,
+          "memory": "256"
+        },
+        "placement_policy": {
+          "constraints": [
+            {
+              "type": "ANTI_AFFINITY",
+              "scope": "NODE",
+              "node_attributes": {
+                "os": ["linux", "windows"],
+                "fault_domain": ["fd1", "fd2"]
+              },
+              "node_partitions": [
+                "gpu",
+                "fast-disk"
+              ],
+              "target_tags": [
+                "hello"
+              ]
+            }
+          ]
+        }
+      }
+    ]
+}
+```
+
+##### GET Response JSON
+GET URL - http://localhost:8088/app/v1/services/hello-world
+
+Note, for an anti-affinity component no more than 1 container will be allocated
+in a specific node. In this example, 3 containers have been requested by
+component "hello". All 3 containers were allocated because the cluster had 3 or
+more NMs. If the cluster had less than 3 NMs then less than 3 containers would
+be allocated. In cases when the number of allocated containers are less than the
+number of requested containers, the component and the service will be in
+non-STABLE state.
+
+```json
+{
+    "name": "hello-world",
+    "version": "1.0.0",
+    "description": "hello world example with anti-affinity",
+    "id": "application_1503963985568_0003",
+    "lifetime": -1,
+    "state": "STABLE",
+    "components": [
+        {
+            "name": "hello",
+            "state": "STABLE",
+            "resource": {
+                "cpus": 1,
+                "memory": "256"
+            },
+            "placement_policy": {
+              "constraints": [
+                {
+                  "type": "ANTI_AFFINITY",
+                  "scope": "NODE",
+                  "node_attributes": {
+                    "os": ["linux", "windows"],
+                    "fault_domain": ["fd1", "fd2"]
+                  },
+                  "node_partitions": [
+                    "gpu",
+                    "fast-disk"
+                  ],
+                  "target_tags": [
+                    "hello"
+                  ]
+                }
+              ]
+            },
+            "configuration": {
+                "properties": {},
+                "env": {},
+                "files": []
+            },
+            "quicklinks": [],
+            "containers": [
+                {
+                    "id": "container_e03_1503963985568_0003_01_000002",
+                    "ip": "10.22.8.143",
+                    "hostname": "ctr-e03-1503963985568-0003-01-000002.example.site",
+                    "state": "READY",
+                    "launch_time": 1504051512412,
+                    "bare_host": "host100.cloud.com",
+                    "component_instance_name": "hello-0"
+                },
+                {
+                    "id": "container_e03_1503963985568_0003_01_000003",
+                    "ip": "10.22.8.144",
+                    "hostname": "ctr-e03-1503963985568-0003-01-000003.example.site",
+                    "state": "READY",
+                    "launch_time": 1504051536450,
+                    "bare_host": "host101.cloud.com",
+                    "component_instance_name": "hello-1"
+                },
+                {
+                    "id": "container_e03_1503963985568_0003_01_000004",
+                    "ip": "10.22.8.145",
+                    "hostname": "ctr-e03-1503963985568-0003-01-000004.example.site",
+                    "state": "READY",
+                    "launch_time": 1504051536450,
+                    "bare_host": "host102.cloud.com",
+                    "component_instance_name": "hello-2"
+                }
+            ],
+            "launch_command": "./start_nginx.sh",
+            "number_of_containers": 1,
+            "run_privileged_container": false
+        }
+    ],
+    "configuration": {
+        "properties": {},
+        "env": {},
+        "files": []
+    },
+    "quicklinks": {}
+}
+```
+


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