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 bi...@apache.org on 2019/02/10 17:25:36 UTC

[hadoop] branch branch-3.1 updated: YARN-8761. Service AM support for decommissioning component instances. Contributed by Billie Rinaldi

This is an automated email from the ASF dual-hosted git repository.

billie pushed a commit to branch branch-3.1
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/branch-3.1 by this push:
     new 102db40  YARN-8761. Service AM support for decommissioning component instances.            Contributed by Billie Rinaldi
102db40 is described below

commit 102db408708195f1c006a0d933a4d5e78cf2874a
Author: Eric Yang <ey...@apache.org>
AuthorDate: Mon Nov 12 19:53:10 2018 -0500

    YARN-8761. Service AM support for decommissioning component instances.
               Contributed by Billie Rinaldi
    
    (cherry picked from commit 4c465f5535054dad2ef0b18128fb115129f6939e)
---
 .../yarn/service/client/ApiServiceClient.java      |  28 ++++
 .../hadoop/yarn/service/webapp/ApiServer.java      |  40 ++++++
 .../YARN-Simplified-V1-API-Layer-For-Services.yaml |   5 +
 .../hadoop/yarn/service/ClientAMProtocol.java      |   6 +
 .../hadoop/yarn/service/ClientAMService.java       |  20 +++
 .../hadoop/yarn/service/api/records/Component.java |  26 ++++
 .../hadoop/yarn/service/client/ServiceClient.java  |  60 ++++++++-
 .../hadoop/yarn/service/component/Component.java   |  64 ++++++++-
 .../yarn/service/component/ComponentEvent.java     |  10 ++
 .../yarn/service/component/ComponentEventType.java |   3 +-
 .../component/instance/ComponentInstance.java      |  18 +--
 .../pb/client/ClientAMProtocolPBClientImpl.java    |  14 ++
 .../pb/service/ClientAMProtocolPBServiceImpl.java  |  13 ++
 .../hadoop/yarn/service/utils/ServiceApiUtil.java  |  56 +++++++-
 .../src/main/proto/ClientAMProtocol.proto          |  11 +-
 .../hadoop/yarn/service/ServiceTestUtils.java      |   2 +-
 .../TestComponentDecommissionInstances.java        | 147 +++++++++++++++++++++
 .../hadoop/yarn/client/cli/ApplicationCLI.java     |  26 +++-
 .../apache/hadoop/yarn/client/cli/TestYarnCLI.java |  15 ++-
 .../hadoop/yarn/client/api/AppAdminClient.java     |  12 ++
 .../src/site/markdown/YarnCommands.md              |   3 +
 .../site/markdown/yarn-service/YarnServiceAPI.md   |   1 +
 22 files changed, 548 insertions(+), 32 deletions(-)

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/ApiServiceClient.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/ApiServiceClient.java
index 621dedb..15c1bab 100644
--- 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/ApiServiceClient.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/ApiServiceClient.java
@@ -745,4 +745,32 @@ public class ApiServiceClient extends AppAdminClient {
     }
     return result;
   }
+
+  @Override
+  public int actionDecommissionInstances(String appName, List<String>
+      componentInstances) throws IOException, YarnException {
+    int result = EXIT_SUCCESS;
+    try {
+      Service service = new Service();
+      service.setName(appName);
+      for (String instance : componentInstances) {
+        String componentName = ServiceApiUtil.parseComponentName(instance);
+        Component component = service.getComponent(componentName);
+        if (component == null) {
+          component = new Component();
+          component.setName(componentName);
+          service.addComponent(component);
+        }
+        component.addDecommissionedInstance(instance);
+      }
+      String buffer = jsonSerDeser.toJson(service);
+      ClientResponse response = getApiClient(getServicePath(appName))
+          .put(ClientResponse.class, buffer);
+      result = processResponse(response);
+    } catch (Exception e) {
+      LOG.error("Fail to decommission instance: ", e);
+      result = EXIT_EXCEPTION_THROWN;
+    }
+    return result;
+  }
 }
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
index 51ad00a..9537e42 100644
--- 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
@@ -469,6 +469,12 @@ public class ApiServer {
           && updateServiceData.getLifetime() > 0) {
         return updateLifetime(appName, updateServiceData, ugi);
       }
+
+      for (Component c : updateServiceData.getComponents()) {
+        if (c.getDecommissionedInstances().size() > 0) {
+          return decommissionInstances(updateServiceData, ugi);
+        }
+      }
     } catch (UndeclaredThrowableException e) {
       return formatResponse(Status.BAD_REQUEST,
           e.getCause().getMessage());
@@ -824,6 +830,40 @@ public class ApiServer {
     });
   }
 
+  private Response decommissionInstances(Service service, UserGroupInformation
+      ugi) throws IOException, InterruptedException {
+    String appName = service.getName();
+    Response response = Response.status(Status.BAD_REQUEST).build();
+
+    List<String> instances = new ArrayList<>();
+    for (Component c : service.getComponents()) {
+      instances.addAll(c.getDecommissionedInstances());
+    }
+    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
+            .actionDecommissionInstances(appName, instances);
+        sc.close();
+        return Integer.valueOf(result);
+      }
+    });
+    if (result == EXIT_SUCCESS) {
+      String message = "Service " + appName + " has successfully " +
+          "decommissioned instances.";
+      LOG.info(message);
+      ServiceStatus status = new ServiceStatus();
+      status.setDiagnostics(message);
+      status.setState(ServiceState.ACCEPTED);
+      response = formatResponse(Status.ACCEPTED, status);
+    }
+    return response;
+  }
+
   private Service getServiceFromClient(UserGroupInformation ugi,
       String serviceName) throws IOException, InterruptedException {
 
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
index d90ae06..c983f51 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-api/src/main/resources/definition/YARN-Simplified-V1-API-Layer-For-Services.yaml
@@ -405,6 +405,11 @@ definitions:
         type: integer
         format: int64
         description: Number of containers for this component (optional). If not specified, the service level global number_of_containers takes effect.
+      decommissioned_instances:
+        type: array
+        items:
+          type: string
+        description: List of decommissioned component instances.
       containers:
         type: array
         description: Containers of a started component. Specifying a value for this attribute for the POST payload raises a validation error. This blob is available only in the GET response of a started service.
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/ClientAMProtocol.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/ClientAMProtocol.java
index 39e7dfa..e43e6fb 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/ClientAMProtocol.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/ClientAMProtocol.java
@@ -23,6 +23,8 @@ import org.apache.hadoop.yarn.proto.ClientAMProtocol.CancelUpgradeRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.CancelUpgradeResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.CompInstancesUpgradeResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.CompInstancesUpgradeRequestProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.DecommissionCompInstancesRequestProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.DecommissionCompInstancesResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetCompInstancesRequestProto;
@@ -65,4 +67,8 @@ public interface ClientAMProtocol {
 
   CancelUpgradeResponseProto cancelUpgrade(
       CancelUpgradeRequestProto request) throws IOException, YarnException;
+
+  DecommissionCompInstancesResponseProto decommissionCompInstances(
+      DecommissionCompInstancesRequestProto request) throws IOException,
+      YarnException;
 }
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/ClientAMService.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/ClientAMService.java
index 47e9829..10423c3 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/ClientAMService.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/ClientAMService.java
@@ -35,6 +35,8 @@ import org.apache.hadoop.yarn.proto.ClientAMProtocol.CancelUpgradeResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.CompInstancesUpgradeRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.CompInstancesUpgradeResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.ComponentCountProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.DecommissionCompInstancesRequestProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.DecommissionCompInstancesResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetCompInstancesRequestProto;
@@ -60,6 +62,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.List;
 
+import static org.apache.hadoop.yarn.service.component.ComponentEventType.DECOMMISSION_INSTANCE;
 import static org.apache.hadoop.yarn.service.component.ComponentEventType.FLEX;
 
 public class ClientAMService extends AbstractService
@@ -220,4 +223,21 @@ public class ClientAMService extends AbstractService
     context.scheduler.getDispatcher().getEventHandler().handle(event);
     return CancelUpgradeResponseProto.newBuilder().build();
   }
+
+  @Override
+  public DecommissionCompInstancesResponseProto decommissionCompInstances(
+      DecommissionCompInstancesRequestProto request)
+      throws IOException, YarnException {
+    if (!request.getCompInstancesList().isEmpty()) {
+      for (String instance : request.getCompInstancesList()) {
+        String componentName = ServiceApiUtil.parseComponentName(instance);
+        ComponentEvent event = new ComponentEvent(componentName,
+            DECOMMISSION_INSTANCE).setInstanceName(instance);
+        context.scheduler.getDispatcher().getEventHandler().handle(event);
+        LOG.info("Decommissioning component {} instance {}", componentName,
+            instance);
+      }
+    }
+    return DecommissionCompInstancesResponseProto.newBuilder().build();
+  }
 }
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/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/api/records/Component.java
index 0481123..9a117cd 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/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/api/records/Component.java
@@ -80,6 +80,10 @@ public class Component implements Serializable {
   @XmlElement(name = "number_of_containers")
   private Long numberOfContainers = null;
 
+  @JsonProperty("decommissioned_instances")
+  @XmlElement(name = "decommissioned_instances")
+  private List<String> decommissionedInstances = new ArrayList<>();
+
   @JsonProperty("run_privileged_container")
   @XmlElement(name = "run_privileged_container")
   private Boolean runPrivilegedContainer = false;
@@ -296,6 +300,28 @@ public class Component implements Serializable {
     this.numberOfContainers = numberOfContainers;
   }
 
+  /**
+   * A list of decommissioned component instances.
+   **/
+  public Component decommissionedInstances(List<String>
+      decommissionedInstances) {
+    this.decommissionedInstances = decommissionedInstances;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A list of decommissioned component instances.")
+  public List<String> getDecommissionedInstances() {
+    return decommissionedInstances;
+  }
+
+  public void setDecommissionedInstances(List<String> decommissionedInstances) {
+    this.decommissionedInstances = decommissionedInstances;
+  }
+
+  public void addDecommissionedInstance(String componentInstanceName) {
+    this.decommissionedInstances.add(componentInstanceName);
+  }
+
   @ApiModelProperty(example = "null", value = "Containers of a started component. Specifying a value for this attribute for the POST payload raises a validation error. This blob is available only in the GET response of a started service.")
   public List<Container> getContainers() {
     return containers;
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/client/ServiceClient.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/client/ServiceClient.java
index 80eb67d..c2bea1e 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/client/ServiceClient.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/client/ServiceClient.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.CancelUpgradeRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.CompInstancesUpgradeRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.ComponentCountProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.DecommissionCompInstancesRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetCompInstancesRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetCompInstancesResponseProto;
@@ -375,6 +376,61 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
   }
 
   @Override
+  public int actionDecommissionInstances(String appName,
+      List<String> componentInstances) throws IOException, YarnException {
+    checkAppExistOnHdfs(appName);
+    Service persistedService = ServiceApiUtil.loadService(fs, appName);
+    if (StringUtils.isEmpty(persistedService.getId())) {
+      throw new YarnException(
+          persistedService.getName() + " appId is null, may be not submitted " +
+              "to YARN yet");
+    }
+    cachedAppInfo.put(persistedService.getName(), new AppInfo(
+        ApplicationId.fromString(persistedService.getId()), persistedService
+        .getKerberosPrincipal().getPrincipalName()));
+
+    for (String instance : componentInstances) {
+      String componentName = ServiceApiUtil.parseComponentName(
+          ServiceApiUtil.parseAndValidateComponentInstanceName(instance,
+              appName, getConfig()));
+      Component component = persistedService.getComponent(componentName);
+      if (component == null) {
+        throw new IllegalArgumentException(instance + " does not exist !");
+      }
+      if (!component.getDecommissionedInstances().contains(instance)) {
+        component.addDecommissionedInstance(instance);
+        component.setNumberOfContainers(Math.max(0, component
+            .getNumberOfContainers() - 1));
+      }
+    }
+    ServiceApiUtil.writeAppDefinition(fs, persistedService);
+
+    ApplicationReport appReport =
+        yarnClient.getApplicationReport(ApplicationId.fromString(
+            persistedService.getId()));
+    if (appReport.getYarnApplicationState() != RUNNING) {
+      String message =
+          persistedService.getName() + " is at " + appReport
+              .getYarnApplicationState() + " state, decommission can only be " +
+              "invoked when service is running";
+      LOG.error(message);
+      throw new YarnException(message);
+    }
+
+    if (StringUtils.isEmpty(appReport.getHost())) {
+      throw new YarnException(persistedService.getName() + " AM hostname is " +
+          "empty");
+    }
+    ClientAMProtocol proxy =
+        createAMProxy(persistedService.getName(), appReport);
+    DecommissionCompInstancesRequestProto.Builder requestBuilder =
+        DecommissionCompInstancesRequestProto.newBuilder();
+    requestBuilder.addAllCompInstances(componentInstances);
+    proxy.decommissionCompInstances(requestBuilder.build());
+    return EXIT_SUCCESS;
+  }
+
+  @Override
   public int actionCleanUp(String appName, String userName) throws
       IOException, YarnException {
     if (cleanUpRegistry(appName, userName)) {
@@ -577,9 +633,7 @@ public class ServiceClient extends AppAdminClient implements SliderExitCodes,
       throw new YarnException("Components " + componentCounts.keySet()
           + " do not exist in app definition.");
     }
-    jsonSerDeser
-        .save(fs.getFileSystem(), ServiceApiUtil.getServiceJsonPath(fs, serviceName),
-            persistedService, true);
+    ServiceApiUtil.writeAppDefinition(fs, persistedService);
 
     ApplicationId appId = getAppId(serviceName);
     if (appId == null) {
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 0972afe..ae52cf1 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
@@ -57,6 +57,7 @@ import org.apache.hadoop.yarn.service.monitor.probe.MonitorUtils;
 import org.apache.hadoop.yarn.service.monitor.probe.Probe;
 import org.apache.hadoop.yarn.service.containerlaunch.ContainerLaunchService;
 import org.apache.hadoop.yarn.service.provider.ProviderUtils;
+import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
 import org.apache.hadoop.yarn.service.utils.ServiceUtils;
 import org.apache.hadoop.yarn.state.InvalidStateTransitionException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
@@ -142,6 +143,9 @@ public class Component implements EventHandler<ComponentEvent> {
           // container recovered on AM restart
           .addTransition(INIT, INIT, CONTAINER_RECOVERED,
               new ContainerRecoveredTransition())
+          // instance decommissioned
+          .addTransition(INIT, INIT, DECOMMISSION_INSTANCE,
+              new DecommissionInstanceTransition())
 
           // container recovered in AM heartbeat
           .addTransition(FLEXING, FLEXING, CONTAINER_RECOVERED,
@@ -161,6 +165,9 @@ public class Component implements EventHandler<ComponentEvent> {
               new FlexComponentTransition())
           .addTransition(FLEXING, EnumSet.of(UPGRADING, FLEXING, STABLE),
               CHECK_STABLE, new CheckStableTransition())
+          // instance decommissioned
+          .addTransition(FLEXING, FLEXING, DECOMMISSION_INSTANCE,
+              new DecommissionInstanceTransition())
 
           // container failed while stable
           .addTransition(STABLE, FLEXING, CONTAINER_COMPLETED,
@@ -173,6 +180,10 @@ public class Component implements EventHandler<ComponentEvent> {
           // For flex down, go to STABLE state
           .addTransition(STABLE, EnumSet.of(STABLE, FLEXING),
               FLEX, new FlexComponentTransition())
+          // instance decommissioned
+          .addTransition(STABLE, STABLE, DECOMMISSION_INSTANCE,
+              new DecommissionInstanceTransition())
+          // upgrade component
           .addTransition(STABLE, UPGRADING, UPGRADE,
               new NeedsUpgradeTransition())
           .addTransition(STABLE, CANCEL_UPGRADING, CANCEL_UPGRADE,
@@ -187,6 +198,9 @@ public class Component implements EventHandler<ComponentEvent> {
               CHECK_STABLE, new CheckStableTransition())
           .addTransition(UPGRADING, UPGRADING, CONTAINER_COMPLETED,
               new CompletedAfterUpgradeTransition())
+          // instance decommissioned
+          .addTransition(UPGRADING, UPGRADING, DECOMMISSION_INSTANCE,
+              new DecommissionInstanceTransition())
 
           .addTransition(CANCEL_UPGRADING, EnumSet.of(CANCEL_UPGRADING, FLEXING,
               STABLE), CHECK_STABLE, new CheckStableTransition())
@@ -194,7 +208,9 @@ public class Component implements EventHandler<ComponentEvent> {
               CONTAINER_COMPLETED, new CompletedAfterUpgradeTransition())
           .addTransition(CANCEL_UPGRADING, FLEXING, CONTAINER_ALLOCATED,
               new ContainerAllocatedTransition())
-
+          // instance decommissioned
+          .addTransition(CANCEL_UPGRADING, CANCEL_UPGRADING,
+              DECOMMISSION_INSTANCE, new DecommissionInstanceTransition())
           .installTopology();
 
   public Component(
@@ -241,6 +257,11 @@ public class Component implements EventHandler<ComponentEvent> {
     ComponentInstanceId id =
         new ComponentInstanceId(instanceIdCounter.getAndIncrement(),
             componentSpec.getName());
+    while (componentSpec.getDecommissionedInstances().contains(id
+        .getCompInstanceName())) {
+      id = new ComponentInstanceId(instanceIdCounter.getAndIncrement(),
+          componentSpec.getName());
+    }
     ComponentInstance instance = new ComponentInstance(this, id);
     compInstances.put(instance.getCompInstanceName(), instance);
     pendingInstances.add(instance);
@@ -377,6 +398,38 @@ public class Component implements EventHandler<ComponentEvent> {
     }
   }
 
+  private static class DecommissionInstanceTransition extends BaseTransition {
+    @Override
+    public void transition(Component component, ComponentEvent event) {
+      String instanceName = event.getInstanceName();
+      String hostnameSuffix = component.getHostnameSuffix();
+      if (instanceName.endsWith(hostnameSuffix)) {
+        instanceName = instanceName.substring(0,
+            instanceName.length() - hostnameSuffix.length());
+      }
+      if (component.getComponentSpec().getDecommissionedInstances()
+          .contains(instanceName)) {
+        LOG.info("Instance {} already decommissioned", instanceName);
+        return;
+      }
+      component.getComponentSpec().addDecommissionedInstance(instanceName);
+      ComponentInstance instance = component.getComponentInstance(instanceName);
+      if (instance == null) {
+        LOG.info("Instance was null for decommissioned instance {}",
+            instanceName);
+        return;
+      }
+      // remove the instance
+      component.compInstances.remove(instance.getCompInstanceName());
+      component.pendingInstances.remove(instance);
+      component.scheduler.getServiceMetrics().containersDesired.decr();
+      component.componentMetrics.containersDesired.decr();
+      component.getComponentSpec().setNumberOfContainers(component
+          .getComponentSpec().getNumberOfContainers() - 1);
+      instance.destroy();
+    }
+  }
+
   private static class ContainerAllocatedTransition extends BaseTransition {
     @Override
     public void transition(Component component, ComponentEvent event) {
@@ -807,10 +860,8 @@ public class Component implements EventHandler<ComponentEvent> {
 
   private void setDesiredContainers(int n) {
     int delta = n - scheduler.getServiceMetrics().containersDesired.value();
-    if (delta > 0) {
+    if (delta != 0) {
       scheduler.getServiceMetrics().containersDesired.incr(delta);
-    } else {
-      scheduler.getServiceMetrics().containersDesired.decr(delta);
     }
     componentMetrics.containersDesired.set(n);
   }
@@ -1202,4 +1253,9 @@ public class Component implements EventHandler<ComponentEvent> {
     RestartPolicyEnum restartPolicyEnum = getComponentSpec().getRestartPolicy();
     return getRestartPolicyHandler(restartPolicyEnum);
   }
+
+  public String getHostnameSuffix() {
+    return ServiceApiUtil.getHostnameSuffix(context.service.getName(),
+        scheduler.getConfig());
+  }
 }
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/ComponentEvent.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/ComponentEvent.java
index 84caa77..7966ac2 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/ComponentEvent.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/ComponentEvent.java
@@ -31,6 +31,7 @@ public class ComponentEvent extends AbstractEvent<ComponentEventType> {
   private final ComponentEventType type;
   private Container container;
   private ComponentInstance instance;
+  private String instanceName;
   private ContainerStatus status;
   private ContainerId containerId;
   private org.apache.hadoop.yarn.service.api.records.Component targetSpec;
@@ -86,6 +87,15 @@ public class ComponentEvent extends AbstractEvent<ComponentEventType> {
     return this;
   }
 
+  public String getInstanceName() {
+    return instanceName;
+  }
+
+  public ComponentEvent setInstanceName(String instanceName) {
+    this.instanceName = instanceName;
+    return this;
+  }
+
   public ContainerStatus getStatus() {
     return status;
   }
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/ComponentEventType.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/ComponentEventType.java
index d211f49..558dc90 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/ComponentEventType.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/ComponentEventType.java
@@ -26,5 +26,6 @@ public enum ComponentEventType {
   CONTAINER_COMPLETED,
   CANCEL_UPGRADE,
   UPGRADE,
-  CHECK_STABLE
+  CHECK_STABLE,
+  DECOMMISSION_INSTANCE
 }
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/instance/ComponentInstance.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/instance/ComponentInstance.java
index e9ffec5..a3aa143 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/instance/ComponentInstance.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/instance/ComponentInstance.java
@@ -21,9 +21,7 @@ package org.apache.hadoop.yarn.service.component.instance;
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.registry.client.api.RegistryConstants;
 import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
-import org.apache.hadoop.registry.client.binding.RegistryUtils;
 import org.apache.hadoop.registry.client.types.ServiceRecord;
 import org.apache.hadoop.registry.client.types.yarn.PersistencePolicies;
 import org.apache.hadoop.util.StringUtils;
@@ -984,21 +982,7 @@ public class ComponentInstance implements EventHandler<ComponentInstanceEvent>,
   }
 
   public String getHostname() {
-    String domain = getComponent().getScheduler().getConfig()
-        .get(RegistryConstants.KEY_DNS_DOMAIN);
-    String hostname;
-    if (domain == null || domain.isEmpty()) {
-      hostname = MessageFormat
-          .format("{0}.{1}.{2}", getCompInstanceName(),
-              getComponent().getContext().service.getName(),
-              RegistryUtils.currentUser());
-    } else {
-      hostname = MessageFormat
-          .format("{0}.{1}.{2}.{3}", getCompInstanceName(),
-              getComponent().getContext().service.getName(),
-              RegistryUtils.currentUser(), domain);
-    }
-    return hostname;
+    return getCompInstanceName() + getComponent().getHostnameSuffix();
   }
 
   @Override
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/impl/pb/client/ClientAMProtocolPBClientImpl.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/impl/pb/client/ClientAMProtocolPBClientImpl.java
index 6f37967..c002569 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/impl/pb/client/ClientAMProtocolPBClientImpl.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/impl/pb/client/ClientAMProtocolPBClientImpl.java
@@ -34,6 +34,8 @@ import org.apache.hadoop.yarn.proto.ClientAMProtocol.CancelUpgradeRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.CancelUpgradeResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.CompInstancesUpgradeResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.CompInstancesUpgradeRequestProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.DecommissionCompInstancesRequestProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.DecommissionCompInstancesResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetCompInstancesRequestProto;
@@ -154,4 +156,16 @@ public class ClientAMProtocolPBClientImpl
     }
     return null;
   }
+
+  @Override
+  public DecommissionCompInstancesResponseProto decommissionCompInstances(
+      DecommissionCompInstancesRequestProto request)
+      throws IOException, YarnException {
+    try {
+      return proxy.decommissionCompInstances(null, request);
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+    }
+    return null;
+  }
 }
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/impl/pb/service/ClientAMProtocolPBServiceImpl.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/impl/pb/service/ClientAMProtocolPBServiceImpl.java
index 071c357..b2aac23 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/impl/pb/service/ClientAMProtocolPBServiceImpl.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/impl/pb/service/ClientAMProtocolPBServiceImpl.java
@@ -25,6 +25,8 @@ import org.apache.hadoop.yarn.proto.ClientAMProtocol.CancelUpgradeRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.CancelUpgradeResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.CompInstancesUpgradeRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.CompInstancesUpgradeResponseProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.DecommissionCompInstancesRequestProto;
+import org.apache.hadoop.yarn.proto.ClientAMProtocol.DecommissionCompInstancesResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsRequestProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.FlexComponentsResponseProto;
 import org.apache.hadoop.yarn.proto.ClientAMProtocol.GetCompInstancesRequestProto;
@@ -129,4 +131,15 @@ public class ClientAMProtocolPBServiceImpl implements ClientAMProtocolPB {
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public DecommissionCompInstancesResponseProto decommissionCompInstances(
+      RpcController controller, DecommissionCompInstancesRequestProto
+      request) throws ServiceException {
+    try {
+      return real.decommissionCompInstances(request);
+    } catch (IOException | YarnException e) {
+      throw new ServiceException(e);
+    }
+  }
 }
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/ServiceApiUtil.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/ServiceApiUtil.java
index 5eb1733..28cb07a 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/ServiceApiUtil.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/ServiceApiUtil.java
@@ -53,6 +53,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
@@ -548,6 +549,13 @@ public class ServiceApiUtil {
     return appJson;
   }
 
+  public static Path writeAppDefinition(SliderFileSystem fs, Service service)
+      throws IOException {
+    Path appJson = getServiceJsonPath(fs, service.getName());
+    jsonSerDeser.save(fs.getFileSystem(), appJson, service, true);
+    return appJson;
+  }
+
   public static List<Container> getLiveContainers(Service service,
       List<String> componentInstances)
       throws YarnException {
@@ -657,9 +665,53 @@ public class ServiceApiUtil {
     return containerNeedUpgrade;
   }
 
-  private static String parseComponentName(String componentInstanceName)
+  public static String getHostnameSuffix(String serviceName, org.apache
+      .hadoop.conf.Configuration conf) {
+    String domain = conf.get(RegistryConstants.KEY_DNS_DOMAIN);
+    String hostnameSuffix;
+    if (domain == null || domain.isEmpty()) {
+      hostnameSuffix = MessageFormat
+          .format(".{0}.{1}", serviceName, RegistryUtils.currentUser());
+    } else {
+      hostnameSuffix = MessageFormat
+          .format(".{0}.{1}.{2}", serviceName,
+              RegistryUtils.currentUser(), domain);
+    }
+    return hostnameSuffix;
+  }
+
+  public static String parseAndValidateComponentInstanceName(String
+      instanceOrHostname, String serviceName, org.apache.hadoop.conf
+      .Configuration conf) throws IllegalArgumentException {
+    int idx = instanceOrHostname.indexOf('.');
+    String hostnameSuffix = getHostnameSuffix(serviceName, conf);
+    if (idx != -1) {
+      if (!instanceOrHostname.endsWith(hostnameSuffix)) {
+        throw new IllegalArgumentException("Specified hostname " +
+            instanceOrHostname + " does not have the expected format " +
+            "componentInstanceName" +
+            hostnameSuffix);
+      }
+      instanceOrHostname = instanceOrHostname.substring(0, instanceOrHostname
+          .length() - hostnameSuffix.length());
+    }
+    idx = instanceOrHostname.indexOf('.');
+    if (idx != -1) {
+      throw new IllegalArgumentException("Specified hostname " +
+          instanceOrHostname + " does not have the expected format " +
+          "componentInstanceName" +
+          hostnameSuffix);
+    }
+    return instanceOrHostname;
+  }
+
+  public static String parseComponentName(String componentInstanceName)
       throws YarnException {
-    int idx = componentInstanceName.lastIndexOf('-');
+    int idx = componentInstanceName.indexOf('.');
+    if (idx != -1) {
+      componentInstanceName = componentInstanceName.substring(0, idx);
+    }
+    idx = componentInstanceName.lastIndexOf('-');
     if (idx == -1) {
       throw new YarnException("Invalid component instance (" +
           componentInstanceName + ") name.");
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/proto/ClientAMProtocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/proto/ClientAMProtocol.proto
index bcf893e..85f9b8f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/proto/ClientAMProtocol.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/proto/ClientAMProtocol.proto
@@ -36,6 +36,8 @@ service ClientAMProtocolService {
     (CompInstancesUpgradeResponseProto);
   rpc getCompInstances(GetCompInstancesRequestProto) returns
     (GetCompInstancesResponseProto);
+  rpc decommissionCompInstances(DecommissionCompInstancesRequestProto)
+    returns (DecommissionCompInstancesResponseProto);
 }
 
 message FlexComponentsRequestProto {
@@ -102,4 +104,11 @@ message GetCompInstancesRequestProto {
 
 message GetCompInstancesResponseProto {
   optional string compInstances = 1;
-}
\ No newline at end of file
+}
+
+message DecommissionCompInstancesRequestProto {
+  repeated string compInstances = 1;
+}
+
+message DecommissionCompInstancesResponseProto {
+}
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/ServiceTestUtils.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/ServiceTestUtils.java
index 58db752..b4859af 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/ServiceTestUtils.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/ServiceTestUtils.java
@@ -246,7 +246,7 @@ public class ServiceTestUtils {
 
     if (yarnCluster == null) {
       yarnCluster =
-          new MiniYARNCluster(TestYarnNativeServices.class.getSimpleName(), 1,
+          new MiniYARNCluster(this.getClass().getSimpleName(), 1,
               numNodeManager, 1, 1);
       yarnCluster.init(conf);
       yarnCluster.start();
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/component/TestComponentDecommissionInstances.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/component/TestComponentDecommissionInstances.java
new file mode 100644
index 0000000..e617410
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/component/TestComponentDecommissionInstances.java
@@ -0,0 +1,147 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.component;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.registry.client.binding.RegistryUtils;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.service.ServiceTestUtils;
+import org.apache.hadoop.yarn.service.api.records.Component;
+import org.apache.hadoop.yarn.service.api.records.Container;
+import org.apache.hadoop.yarn.service.api.records.Service;
+import org.apache.hadoop.yarn.service.api.records.ServiceState;
+import org.apache.hadoop.yarn.service.client.ServiceClient;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+
+/**
+ * Test decommissioning component instances.
+ */
+public class TestComponentDecommissionInstances extends ServiceTestUtils {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestComponentDecommissionInstances.class);
+
+  private static final String APP_NAME = "test-decommission";
+  private static final String COMPA = "compa";
+
+  @Rule
+  public TemporaryFolder tmpFolder = new TemporaryFolder();
+
+  @Before
+  public void setup() throws Exception {
+    File tmpYarnDir = new File("target", "tmp");
+    FileUtils.deleteQuietly(tmpYarnDir);
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    shutdown();
+  }
+
+  @Test
+  public void testDecommissionInstances() throws Exception {
+    setupInternal(3);
+    ServiceClient client = createClient(getConf());
+    Service exampleApp = new Service();
+    exampleApp.setName(APP_NAME);
+    exampleApp.setVersion("v1");
+    Component comp = createComponent(COMPA, 6L, "sleep 1000");
+    exampleApp.addComponent(comp);
+    client.actionCreate(exampleApp);
+    waitForServiceToBeStable(client, exampleApp);
+
+    checkInstances(client, COMPA + "-0", COMPA + "-1", COMPA + "-2",
+        COMPA + "-3", COMPA + "-4", COMPA + "-5");
+    client.actionDecommissionInstances(APP_NAME, Arrays.asList(COMPA + "-1",
+        COMPA + "-5"));
+    waitForNumInstances(client, 4);
+    checkInstances(client, COMPA + "-0", COMPA + "-2", COMPA + "-3",
+        COMPA + "-4");
+
+    // Stop and start service
+    client.actionStop(APP_NAME);
+    waitForServiceToBeInState(client, exampleApp, ServiceState.STOPPED);
+    client.actionStart(APP_NAME);
+    waitForServiceToBeStable(client, exampleApp);
+    checkInstances(client, COMPA + "-0", COMPA + "-2", COMPA + "-3",
+        COMPA + "-4");
+
+    Map<String, String> compCounts = new HashMap<>();
+    compCounts.put(COMPA, "5");
+    client.actionFlex(APP_NAME, compCounts);
+    waitForNumInstances(client, 5);
+    checkInstances(client, COMPA + "-0", COMPA + "-2", COMPA + "-3",
+        COMPA + "-4", COMPA + "-6");
+
+    client.actionDecommissionInstances(APP_NAME, Arrays.asList(COMPA + "-0."
+            + APP_NAME + "." + RegistryUtils.currentUser()));
+    waitForNumInstances(client, 4);
+    checkInstances(client, COMPA + "-2", COMPA + "-3",
+        COMPA + "-4", COMPA + "-6");
+  }
+
+  private static void waitForNumInstances(ServiceClient client, int
+      expectedInstances) throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(() -> {
+      try {
+        Service retrievedApp = client.getStatus(APP_NAME);
+        return retrievedApp.getComponent(COMPA).getContainers().size() ==
+            expectedInstances && retrievedApp.getState() == ServiceState.STABLE;
+      } catch (Exception e) {
+        e.printStackTrace();
+        return false;
+      }
+    }, 2000, 200000);
+  }
+
+  private static void checkInstances(ServiceClient client, String... instances)
+      throws IOException, YarnException {
+    Service service = client.getStatus(APP_NAME);
+    Component component = service.getComponent(COMPA);
+    Assert.assertEquals("Service state should be STABLE", ServiceState.STABLE,
+        service.getState());
+    Assert.assertEquals(instances.length + " containers are expected to be " +
+        "running", instances.length, component.getContainers().size());
+    Set<String> existingInstances = new HashSet<>();
+    for (Container cont : component.getContainers()) {
+      existingInstances.add(cont.getComponentInstanceName());
+    }
+    Assert.assertEquals(instances.length + " instances are expected to be " +
+        "running", instances.length, existingInstances.size());
+    for (String instance : instances) {
+      Assert.assertTrue("Expected instance did not exist " + instance,
+          existingInstances.contains(instance));
+    }
+  }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
index b0e12bc..480ea23 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
@@ -99,6 +99,7 @@ public class ApplicationCLI extends YarnCLI {
   public static final String DESTROY_CMD = "destroy";
   public static final String FLEX_CMD = "flex";
   public static final String COMPONENT = "component";
+  public static final String DECOMMISSION = "decommission";
   public static final String ENABLE_FAST_LAUNCH = "enableFastLaunch";
   public static final String UPGRADE_CMD = "upgrade";
   public static final String UPGRADE_EXPRESS = "express";
@@ -239,6 +240,10 @@ public class ApplicationCLI extends YarnCLI {
           "yarn-service. If ID is provided, the appType will be looked up. " +
           "Supports -appTypes option to specify which client implementation " +
           "to use.");
+      opts.addOption(DECOMMISSION, true, "Decommissions component " +
+          "instances for an application / long-running service. Requires " +
+          "-instances option. Supports -appTypes option to specify which " +
+          "client implementation to use.");
       opts.addOption(COMPONENT, true, "Works with -flex option to change " +
           "the number of components/containers running for an application / " +
           "long-running service. Supports absolute or relative changes, such " +
@@ -258,9 +263,12 @@ public class ApplicationCLI extends YarnCLI {
           "application specification file.");
       opts.addOption(COMPONENT_INSTS, true, "Works with -upgrade option to " +
           "trigger the upgrade of specified component instances of the " +
-          "application.");
+          "application. Also works with -decommission option to decommission " +
+          "specified component instances. Multiple instances should be " +
+          "separated by commas.");
       opts.addOption(COMPONENTS, true, "Works with -upgrade option to " +
-          "trigger the upgrade of specified components of the application.");
+          "trigger the upgrade of specified components of the application. " +
+          "Multiple components should be separated by commas.");
       opts.addOption(UPGRADE_FINALIZE, false, "Works with -upgrade option to " +
           "finalize the upgrade.");
       opts.addOption(UPGRADE_AUTO_FINALIZE, false, "Works with -upgrade and " +
@@ -290,6 +298,8 @@ public class ApplicationCLI extends YarnCLI {
       opts.getOption(COMPONENTS).setArgName("Components");
       opts.getOption(COMPONENTS).setValueSeparator(',');
       opts.getOption(COMPONENTS).setArgs(Option.UNLIMITED_VALUES);
+      opts.getOption(DECOMMISSION).setArgName("Application Name");
+      opts.getOption(DECOMMISSION).setArgs(1);
     } else if (title != null && title.equalsIgnoreCase(APPLICATION_ATTEMPT)) {
       opts.addOption(STATUS_CMD, true,
           "Prints the status of the application attempt.");
@@ -708,6 +718,18 @@ public class ApplicationCLI extends YarnCLI {
         }
         return client.actionCancelUpgrade(appName);
       }
+    } else if (cliParser.hasOption(DECOMMISSION)) {
+      if (!cliParser.hasOption(COMPONENT_INSTS) ||
+          hasAnyOtherCLIOptions(cliParser, opts, DECOMMISSION, COMPONENT_INSTS,
+              APP_TYPE_CMD)) {
+        printUsage(title, opts);
+        return exitCode;
+      }
+      String[] instances = cliParser.getOptionValues(COMPONENT_INSTS);
+      String[] appNameAndType = getAppNameAndType(cliParser, DECOMMISSION);
+      return AppAdminClient.createAppAdminClient(appNameAndType[1], getConf())
+          .actionDecommissionInstances(appNameAndType[0],
+              Arrays.asList(instances));
     } else {
       syserr.println("Invalid Command Usage : ");
       printUsage(title, opts);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
index c6429ec..0c879ae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
@@ -2149,6 +2149,15 @@ public class TestYarnCLI {
     pw.println(" -components <Components>                 Works with -upgrade option to");
     pw.println("                                          trigger the upgrade of specified");
     pw.println("                                          components of the application.");
+    pw.println("                                          Multiple components should be");
+    pw.println("                                          separated by commas.");
+    pw.println(" -decommission <Application Name>         Decommissions component");
+    pw.println("                                          instances for an application /");
+    pw.println("                                          long-running service. Requires");
+    pw.println("                                          -instances option. Supports");
+    pw.println("                                          -appTypes option to specify");
+    pw.println("                                          which client implementation to");
+    pw.println("                                          use.");
     pw.println(" -destroy <Application Name>              Destroys a saved application");
     pw.println("                                          specification and removes all");
     pw.println("                                          application data permanently.");
@@ -2190,7 +2199,11 @@ public class TestYarnCLI {
     pw.println(" -instances <Component Instances>         Works with -upgrade option to");
     pw.println("                                          trigger the upgrade of specified");
     pw.println("                                          component instances of the");
-    pw.println("                                          application.");
+    pw.println("                                          application. Also works with");
+    pw.println("                                          -decommission option to");
+    pw.println("                                          decommission specified component");
+    pw.println("                                          instances. Multiple instances");
+    pw.println("                                          should be separated by commas.");
     pw.println(" -kill <Application ID>                   Kills the application. Set of");
     pw.println("                                          applications can be provided");
     pw.println("                                          separated with space");
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
index df11ffd..b0d737a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/AppAdminClient.java
@@ -313,4 +313,16 @@ public abstract class AppAdminClient extends CompositeService {
   @Unstable
   public abstract int actionCancelUpgrade(String appName) throws IOException,
       YarnException;
+
+  /**
+   * Decommission component instances of a long running service.
+   *
+   * @param appName            the name of the application.
+   * @param componentInstances the name of the component instances.
+   */
+  @Public
+  @Unstable
+  public abstract int actionDecommissionInstances(String appName,
+      List<String> componentInstances) throws IOException, YarnException;
+
 }
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
index 67163c4..e17538c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnCommands.md
@@ -50,10 +50,13 @@ Usage: `yarn app [options] `
 | -appTypes \<Types\> | Works with -list to filter applications based on input comma-separated list of application types. |
 | -changeQueue \<Queue Name\> | Moves application to a new queue. ApplicationId can be passed using 'appId' option. 'movetoqueue' command is deprecated, this new command 'changeQueue' performs same functionality. |
 | -component \<Component Name\> \<Count\> | Works with -flex option to change the number of components/containers running for an application / long-running service. Supports absolute or relative changes, such as +1, 2, or -3. |
+| -components \<Components\> | Works with -upgrade option to trigger the upgrade of specified components of the application. Multiple components should be separated by commas. |
+| -decommission \<Application Name\> | Decommissions component instances for an application / long-running service. Requires -instances option. Supports -appTypes option to specify which client implementation to use. |
 | -destroy \<Application Name\> | Destroys a saved application specification and removes all application data permanently. Supports -appTypes option to specify which client implementation to use. |
 | -enableFastLaunch | Uploads AM dependencies to HDFS to make future launches faster. Supports -appTypes option to specify which client implementation to use. |
 | -flex \<Application Name or ID\> | Changes number of running containers for a component of an application / long-running service. Requires -component option. If name is provided, appType must be provided unless it is the default yarn-service. If ID is provided, the appType will be looked up. Supports -appTypes option to specify which client implementation to use. |
 | -help | Displays help for all commands. |
+| -instances \<Component Instances\> | Works with -upgrade option to trigger the upgrade of specified component instances of the application. Also works with -decommission option to decommission specified component instances. Multiple instances should be separated by commas. |
 | -kill \<Application ID\> | Kills the application. Set of applications can be provided separated with space |
 | -launch \<Application Name\> \<File Name\> | Launches application from specification file (saves specification and starts application). Options -updateLifetime and -changeQueue can be specified to alter the values provided in the file. Supports -appTypes option to specify which client implementation to use. |
 | -list | List applications. Supports optional use of -appTypes to filter applications based on application type, -appStates to filter applications based on application state and -appTags to filter applications based on application tag. |
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
index fa2c548..b4641b3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/yarn-service/YarnServiceAPI.md
@@ -231,6 +231,7 @@ One or more components of the service. If the service is HBase say, then the com
 |launch_command|The custom launch command of this component (optional for DOCKER component, required otherwise). When specified at the component level, it overrides the value specified at the global level (if any). If docker image supports ENTRYPOINT, launch_command is delimited by comma(,) instead of space.|false|string||
 |resource|Resource of this component (optional). If not specified, the service level global resource takes effect.|false|Resource||
 |number_of_containers|Number of containers for this component (optional). If not specified, the service level global number_of_containers takes effect.|false|integer (int64)||
+|decommissioned_instances|List of decommissioned component instances.|false|string array||
 |containers|Containers of a started component. Specifying a value for this attribute for the POST payload raises a validation error. This blob is available only in the GET response of a started service.|false|Container array||
 |run_privileged_container|Run all containers of this component in privileged mode (YARN-4262).|false|boolean||
 |placement_policy|Advanced scheduling and placement policies for all containers of this component.|false|PlacementPolicy||


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