You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2017/09/25 23:39:08 UTC

[69/86] [abbrv] hadoop git commit: YARN-7091. Rename application to service in yarn-native-services. Contributed by Jian He

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.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/ServiceScheduler.java
new file mode 100644
index 0000000..fb2fd16
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/ServiceScheduler.java
@@ -0,0 +1,654 @@
+/**
+ * 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;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.api.RegistryOperationsFactory;
+import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
+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.registry.client.types.yarn.YarnRegistryAttributes;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.service.CompositeService;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.UpdatedContainer;
+import org.apache.hadoop.yarn.client.api.AMRMClient;
+import org.apache.hadoop.yarn.client.api.TimelineV2Client;
+import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
+import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.event.AsyncDispatcher;
+import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.service.api.ServiceApiConstants;
+import org.apache.hadoop.yarn.service.api.records.Service;
+import org.apache.hadoop.yarn.service.api.records.ConfigFile;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstance;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEvent;
+import org.apache.hadoop.yarn.service.component.instance.ComponentInstanceEventType;
+import org.apache.hadoop.yarn.service.component.Component;
+import org.apache.hadoop.yarn.service.component.ComponentEvent;
+import org.apache.hadoop.yarn.service.component.ComponentEventType;
+import org.apache.hadoop.yarn.service.conf.YarnServiceConstants;
+import org.apache.hadoop.yarn.service.containerlaunch.ContainerLaunchService;
+import org.apache.hadoop.yarn.service.provider.ProviderUtils;
+import org.apache.hadoop.yarn.service.registry.YarnRegistryViewForProviders;
+import org.apache.hadoop.yarn.service.timelineservice.ServiceMetricsSink;
+import org.apache.hadoop.yarn.service.timelineservice.ServiceTimelinePublisher;
+import org.apache.hadoop.yarn.service.utils.ServiceApiUtil;
+import org.apache.hadoop.yarn.service.utils.ServiceRegistryUtils;
+import org.apache.hadoop.yarn.util.BoundedAppender;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.text.MessageFormat;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.fs.FileSystem.FS_DEFAULT_NAME_KEY;
+import static org.apache.hadoop.registry.client.api.RegistryConstants.*;
+import static org.apache.hadoop.yarn.service.api.ServiceApiConstants.*;
+import static org.apache.hadoop.yarn.service.component.ComponentEventType.*;
+
+/**
+ *
+ */
+public class ServiceScheduler extends CompositeService {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ServiceScheduler.class);
+  private Service app;
+
+  // component_name -> component
+  private final Map<String, Component> componentsByName =
+      new ConcurrentHashMap<>();
+
+  // id - > component
+  protected final Map<Long, Component> componentsById =
+      new ConcurrentHashMap<>();
+
+  private final Map<ContainerId, ComponentInstance> liveInstances =
+      new ConcurrentHashMap<>();
+
+  private ServiceMetrics serviceMetrics;
+
+  private ServiceTimelinePublisher serviceTimelinePublisher;
+
+  // Global diagnostics that will be reported to RM on eRxit.
+  // The unit the number of characters. This will be limited to 64 * 1024
+  // characters.
+  private BoundedAppender diagnostics = new BoundedAppender(64 * 1024);
+
+  // A cache for loading config files from remote such as hdfs
+  public LoadingCache<ConfigFile, Object> configFileCache = null;
+
+  public ScheduledExecutorService executorService;
+  public Map<String, String> globalTokens = new HashMap<>();
+
+  private AMRMClientAsync<AMRMClient.ContainerRequest> amRMClient;
+  private NMClientAsync nmClient;
+  private AsyncDispatcher dispatcher;
+  AsyncDispatcher compInstanceDispatcher;
+  private YarnRegistryViewForProviders yarnRegistryOperations;
+  private ServiceContext context;
+  private ContainerLaunchService containerLaunchService;
+
+  public ServiceScheduler(ServiceContext context) {
+    super(context.service.getName());
+    this.context = context;
+  }
+
+  public void buildInstance(ServiceContext context, Configuration configuration)
+      throws YarnException {
+    app = context.service;
+    executorService = Executors.newScheduledThreadPool(10);
+    RegistryOperations registryClient = RegistryOperationsFactory
+        .createInstance("ServiceScheduler", configuration);
+    addIfService(registryClient);
+    yarnRegistryOperations =
+        createYarnRegistryOperations(context, registryClient);
+
+    // register metrics
+    serviceMetrics = ServiceMetrics
+        .register(app.getName(), "Metrics for service");
+    serviceMetrics.tag("type", "Metrics type [component or service]", "service");
+    serviceMetrics.tag("appId", "Service id for service", app.getId());
+
+    amRMClient = createAMRMClient();
+    addIfService(amRMClient);
+
+    nmClient = createNMClient();
+    addIfService(nmClient);
+
+    dispatcher = new AsyncDispatcher("Component  dispatcher");
+    dispatcher.register(ComponentEventType.class,
+        new ComponentEventHandler());
+    dispatcher.setDrainEventsOnStop();
+    addIfService(dispatcher);
+
+    compInstanceDispatcher =
+        new AsyncDispatcher("CompInstance dispatcher");
+    compInstanceDispatcher.register(ComponentInstanceEventType.class,
+        new ComponentInstanceEventHandler());
+    addIfService(compInstanceDispatcher);
+    containerLaunchService = new ContainerLaunchService(context.fs);
+    addService(containerLaunchService);
+
+    if (YarnConfiguration.timelineServiceV2Enabled(configuration)) {
+      TimelineV2Client timelineClient = TimelineV2Client
+          .createTimelineClient(context.attemptId.getApplicationId());
+      amRMClient.registerTimelineV2Client(timelineClient);
+      serviceTimelinePublisher = new ServiceTimelinePublisher(timelineClient);
+      addService(serviceTimelinePublisher);
+      DefaultMetricsSystem.instance().register("ServiceMetricsSink",
+          "For processing metrics to ATS",
+          new ServiceMetricsSink(serviceTimelinePublisher));
+      LOG.info("Timeline v2 is enabled.");
+    }
+
+    initGlobalTokensForSubstitute(context);
+    //substitute quicklinks
+    ProviderUtils.substituteMapWithTokens(app.getQuicklinks(), globalTokens);
+    createConfigFileCache(context.fs.getFileSystem());
+
+    createAllComponents();
+  }
+
+  protected YarnRegistryViewForProviders createYarnRegistryOperations(
+      ServiceContext context, RegistryOperations registryClient) {
+    return new YarnRegistryViewForProviders(registryClient,
+        RegistryUtils.currentUser(), YarnServiceConstants.APP_TYPE, app.getName(),
+        context.attemptId);
+  }
+
+  protected NMClientAsync createNMClient() {
+    return NMClientAsync.createNMClientAsync(new NMClientCallback());
+  }
+
+  protected AMRMClientAsync<AMRMClient.ContainerRequest> createAMRMClient() {
+    return AMRMClientAsync
+        .createAMRMClientAsync(1000, new AMRMClientCallback());
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) throws Exception {
+    try {
+      buildInstance(context, conf);
+    } catch (YarnException e) {
+      throw new YarnRuntimeException(e);
+    }
+    super.serviceInit(conf);
+  }
+
+  @Override
+  public void serviceStop() throws Exception {
+    LOG.info("Stopping service scheduler");
+
+    if (executorService != null) {
+      executorService.shutdownNow();
+    }
+
+    DefaultMetricsSystem.shutdown();
+    if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
+      serviceTimelinePublisher
+          .serviceAttemptUnregistered(context, diagnostics.toString());
+    }
+    // Cleanup each component instance. no need to release containers as
+    // they will be automatically released by RM
+    for (ComponentInstance instance : liveInstances.values()) {
+      instance.cleanupRegistryAndCompHdfsDir();
+    }
+    String msg = diagnostics.toString()
+        + "Navigate to the failed component for more details.";
+    amRMClient
+        .unregisterApplicationMaster(FinalApplicationStatus.ENDED, msg, "");
+    LOG.info("Service " + app.getName()
+        + " unregistered with RM, with attemptId = " + context.attemptId
+        + ", diagnostics = " + diagnostics);
+    super.serviceStop();
+  }
+
+  @Override
+  public void serviceStart() throws Exception {
+    super.serviceStart();
+    InetSocketAddress bindAddress = context.clientAMService.getBindAddress();
+    RegisterApplicationMasterResponse response = amRMClient
+        .registerApplicationMaster(bindAddress.getHostName(),
+            bindAddress.getPort(), "N/A");
+    if (response.getClientToAMTokenMasterKey() != null
+        && response.getClientToAMTokenMasterKey().remaining() != 0) {
+      context.secretManager
+          .setMasterKey(response.getClientToAMTokenMasterKey().array());
+    }
+    registerServiceInstance(context.attemptId, app);
+
+    //TODO handle containers recover
+  }
+
+  private void recover() {
+
+  }
+
+  private void initGlobalTokensForSubstitute(ServiceContext context) {
+    // ZK
+    globalTokens.put(ServiceApiConstants.CLUSTER_ZK_QUORUM, getConfig()
+        .getTrimmed(KEY_REGISTRY_ZK_QUORUM, DEFAULT_REGISTRY_ZK_QUORUM));
+    String user = null;
+    try {
+      user = UserGroupInformation.getCurrentUser().getShortUserName();
+    } catch (IOException e) {
+      LOG.error("Failed to get user.", e);
+    }
+    globalTokens
+        .put(SERVICE_ZK_PATH, ServiceRegistryUtils.mkClusterPath(user, app.getName()));
+
+    globalTokens.put(ServiceApiConstants.USER, user);
+    String dnsDomain = getConfig().getTrimmed(KEY_DNS_DOMAIN);
+    if (dnsDomain != null && !dnsDomain.isEmpty()) {
+      globalTokens.put(ServiceApiConstants.DOMAIN, dnsDomain);
+    }
+    // HDFS
+    String clusterFs = getConfig().getTrimmed(FS_DEFAULT_NAME_KEY);
+    if (clusterFs != null && !clusterFs.isEmpty()) {
+      globalTokens.put(ServiceApiConstants.CLUSTER_FS_URI, clusterFs);
+      globalTokens.put(ServiceApiConstants.CLUSTER_FS_HOST,
+          URI.create(clusterFs).getHost());
+    }
+    globalTokens.put(SERVICE_HDFS_DIR, context.serviceHdfsDir);
+    // service name
+    globalTokens.put(SERVICE_NAME_LC, app.getName().toLowerCase());
+    globalTokens.put(SERVICE_NAME, app.getName());
+  }
+
+  private void createConfigFileCache(final FileSystem fileSystem) {
+    this.configFileCache =
+        CacheBuilder.newBuilder().expireAfterAccess(10, TimeUnit.MINUTES)
+            .build(new CacheLoader<ConfigFile, Object>() {
+              @Override public Object load(ConfigFile key) throws Exception {
+                switch (key.getType()) {
+                case HADOOP_XML:
+                  try (FSDataInputStream input = fileSystem
+                      .open(new Path(key.getSrcFile()))) {
+                    org.apache.hadoop.conf.Configuration confRead =
+                        new org.apache.hadoop.conf.Configuration(false);
+                    confRead.addResource(input);
+                    Map<String, String> map = new HashMap<>(confRead.size());
+                    for (Map.Entry<String, String> entry : confRead) {
+                      map.put(entry.getKey(), entry.getValue());
+                    }
+                    return map;
+                  }
+                case TEMPLATE:
+                  try (FSDataInputStream fileInput = fileSystem
+                      .open(new Path(key.getSrcFile()))) {
+                    return IOUtils.toString(fileInput);
+                  }
+                default:
+                  return null;
+                }
+              }
+            });
+    context.configCache = configFileCache;
+  }
+
+  private void registerServiceInstance(ApplicationAttemptId attemptId,
+      Service service) throws IOException {
+    LOG.info("Registering " + attemptId + ", " + service.getName()
+        + " into registry");
+    ServiceRecord serviceRecord = new ServiceRecord();
+    serviceRecord.set(YarnRegistryAttributes.YARN_ID,
+        attemptId.getApplicationId().toString());
+    serviceRecord.set(YarnRegistryAttributes.YARN_PERSISTENCE,
+        PersistencePolicies.APPLICATION);
+    serviceRecord.description = "Yarn Service Master";
+
+    serviceRecord.addExternalEndpoint(RegistryTypeUtils
+        .ipcEndpoint("classpath:org.apache.hadoop.yarn.service.appmaster.ipc",
+            context.clientAMService.getBindAddress()));
+
+    // set any provided attributes
+    setUserProvidedServiceRecordAttributes(service.getConfiguration(),
+        serviceRecord);
+
+    executorService.submit(new Runnable() {
+      @Override public void run() {
+        try {
+          yarnRegistryOperations.registerSelf(serviceRecord, true);
+          LOG.info("Registered service under {}; absolute path {}",
+              yarnRegistryOperations.getSelfRegistrationPath(),
+              yarnRegistryOperations.getAbsoluteSelfRegistrationPath());
+          boolean isFirstAttempt = 1 == attemptId.getAttemptId();
+          // delete the children in case there are any and this is an AM startup.
+          // just to make sure everything underneath is purged
+          if (isFirstAttempt) {
+            yarnRegistryOperations.deleteChildren(
+                yarnRegistryOperations.getSelfRegistrationPath(), true);
+          }
+        } catch (IOException e) {
+          LOG.error(
+              "Failed to register app " + app.getName() + " in registry");
+        }
+      }
+    });
+    if (YarnConfiguration.timelineServiceV2Enabled(getConfig())) {
+      serviceTimelinePublisher.serviceAttemptRegistered(app, getConfig());
+    }
+  }
+
+  private void setUserProvidedServiceRecordAttributes(
+      org.apache.hadoop.yarn.service.api.records.Configuration conf, ServiceRecord record) {
+    String prefix = "service.record.attribute";
+    for (Map.Entry<String, String> entry : conf.getProperties().entrySet()) {
+      if (entry.getKey().startsWith(prefix)) {
+        String key = entry.getKey().substring(prefix.length() + 1);
+        record.set(key, entry.getValue().trim());
+      }
+    }
+  }
+
+  private void createAllComponents() {
+    long allocateId = 0;
+
+    // sort components by dependencies
+    Collection<org.apache.hadoop.yarn.service.api.records.Component> sortedComponents =
+        ServiceApiUtil.sortByDependencies(app.getComponents());
+
+    for (org.apache.hadoop.yarn.service.api.records.Component compSpec : sortedComponents) {
+      Component component = new Component(compSpec, allocateId, context);
+      componentsById.put(allocateId, component);
+      componentsByName.put(component.getName(), component);
+      allocateId++;
+
+      // Trigger the component without dependencies
+      if (component.areDependenciesReady()) {
+        ComponentEvent event = new ComponentEvent(compSpec.getName(), FLEX)
+            .setDesired(compSpec.getNumberOfContainers());
+        component.handle(event);
+      }
+    }
+  }
+
+  private final class ComponentEventHandler
+      implements EventHandler<ComponentEvent> {
+    @Override
+    public void handle(ComponentEvent event) {
+      Component component = componentsByName.get(event.getName());
+
+      if (component == null) {
+        LOG.error("No component exists for " + event.getName());
+        return;
+      }
+      try {
+        component.handle(event);
+      } catch (Throwable t) {
+        LOG.error(MessageFormat
+            .format("[COMPONENT {0}]: Error in handling event type {1}",
+                component.getName(), event.getType()), t);
+      }
+    }
+  }
+
+  private final class ComponentInstanceEventHandler
+      implements EventHandler<ComponentInstanceEvent> {
+    @Override
+    public void handle(ComponentInstanceEvent event) {
+      ComponentInstance instance =
+          liveInstances.get(event.getContainerId());
+      if (instance == null) {
+        LOG.error("No component instance exists for " + event.getContainerId());
+        return;
+      }
+      try {
+        instance.handle(event);
+      } catch (Throwable t) {
+        LOG.error(instance.getCompInstanceId() +
+            ": Error in handling event type " + event.getType(), t);
+      }
+    }
+  }
+
+  class AMRMClientCallback extends AMRMClientAsync.AbstractCallbackHandler {
+
+    @Override
+    public void onContainersAllocated(List<Container> containers) {
+      LOG.info(containers.size() + " containers allocated. ");
+      for (Container container : containers) {
+        Component comp = componentsById.get(container.getAllocationRequestId());
+        ComponentEvent event =
+            new ComponentEvent(comp.getName(), CONTAINER_ALLOCATED)
+                .setContainer(container);
+        dispatcher.getEventHandler().handle(event);
+        LOG.info("[COMPONENT {}]: {} outstanding container requests.",
+            comp.getName(),
+            amRMClient.getMatchingRequests(container.getAllocationRequestId()).size());
+        // remove the corresponding request
+        Collection<AMRMClient.ContainerRequest> collection = amRMClient
+            .getMatchingRequests(container.getAllocationRequestId());
+        if (collection.iterator().hasNext()) {
+          AMRMClient.ContainerRequest request = collection.iterator().next();
+          amRMClient.removeContainerRequest(request);
+        }
+
+      }
+    }
+
+    @Override
+    public void onContainersCompleted(List<ContainerStatus> statuses) {
+      for (ContainerStatus status : statuses) {
+        ContainerId containerId = status.getContainerId();
+        ComponentInstance instance = liveInstances.get(status.getContainerId());
+        if (instance == null) {
+          LOG.error(
+              "Container {} Completed. No component instance exists. exitStatus={}. diagnostics={} ",
+              containerId, status.getExitStatus(), status.getDiagnostics());
+          return;
+        }
+        ComponentEvent event =
+            new ComponentEvent(instance.getCompName(), CONTAINER_COMPLETED)
+                .setStatus(status).setInstance(instance);
+        dispatcher.getEventHandler().handle(event);
+      }
+    }
+
+    @Override
+    public void onContainersUpdated(List<UpdatedContainer> containers) {
+    }
+
+    @Override public void onShutdownRequest() {
+      //Was used for non-work-preserving restart in YARN, should be deprecated.
+    }
+
+    @Override public void onNodesUpdated(List<NodeReport> updatedNodes) {
+      StringBuilder str = new StringBuilder();
+      str.append("Nodes updated info: ").append(System.lineSeparator());
+      for (NodeReport report : updatedNodes) {
+        str.append(report.getNodeId()).append(", state = ")
+            .append(report.getNodeState()).append(", healthDiagnostics = ")
+            .append(report.getHealthReport()).append(System.lineSeparator());
+      }
+      LOG.warn(str.toString());
+    }
+
+    @Override public float getProgress() {
+      // get running containers over desired containers
+      long total = 0;
+      for (org.apache.hadoop.yarn.service.api.records.Component component : app
+          .getComponents()) {
+        total += component.getNumberOfContainers();
+      }
+      // Probably due to user flexed down to 0
+      if (total == 0) {
+        return 100;
+      }
+      return Math.max((float) liveInstances.size() / total * 100, 100);
+    }
+
+    @Override public void onError(Throwable e) {
+      LOG.error("Error in AMRMClient callback handler ", e);
+    }
+  }
+
+
+  private class NMClientCallback extends NMClientAsync.AbstractCallbackHandler {
+
+    @Override public void onContainerStarted(ContainerId containerId,
+        Map<String, ByteBuffer> allServiceResponse) {
+      ComponentInstance instance = liveInstances.get(containerId);
+      if (instance == null) {
+        LOG.error("No component instance exists for " + containerId);
+        return;
+      }
+      ComponentEvent event =
+          new ComponentEvent(instance.getCompName(), CONTAINER_STARTED)
+              .setInstance(instance);
+      dispatcher.getEventHandler().handle(event);
+    }
+
+    @Override public void onContainerStatusReceived(ContainerId containerId,
+        ContainerStatus containerStatus) {
+
+    }
+
+    @Override public void onContainerStopped(ContainerId containerId) {
+
+    }
+
+    @Override
+    public void onStartContainerError(ContainerId containerId, Throwable t) {
+      ComponentInstance instance = liveInstances.get(containerId);
+      if (instance == null) {
+        LOG.error("No component instance exists for " + containerId);
+        return;
+      }
+      amRMClient.releaseAssignedContainer(containerId);
+      // After container released, it'll get CONTAINER_COMPLETED event from RM
+      // automatically which will trigger stopping COMPONENT INSTANCE
+    }
+
+    @Override public void onContainerResourceIncreased(ContainerId containerId,
+        Resource resource) {
+
+    }
+
+    @Override public void onGetContainerStatusError(ContainerId containerId,
+        Throwable t) {
+
+    }
+
+    @Override
+    public void onIncreaseContainerResourceError(ContainerId containerId,
+        Throwable t) {
+
+    }
+
+    @Override
+    public void onStopContainerError(ContainerId containerId, Throwable t) {
+
+    }
+  }
+
+  public ServiceMetrics getServiceMetrics() {
+    return serviceMetrics;
+  }
+
+  public AMRMClientAsync<AMRMClient.ContainerRequest> getAmRMClient() {
+    return amRMClient;
+  }
+
+  public NMClientAsync getNmClient() {
+    return nmClient;
+  }
+
+  public void addLiveCompInstance(ContainerId containerId,
+      ComponentInstance instance) {
+    liveInstances.put(containerId, instance);
+  }
+
+  public void removeLiveCompInstance(ContainerId containerId) {
+    liveInstances.remove(containerId);
+  }
+
+  public AsyncDispatcher getCompInstanceDispatcher() {
+    return compInstanceDispatcher;
+  }
+
+  public YarnRegistryViewForProviders getYarnRegistryOperations() {
+    return yarnRegistryOperations;
+  }
+
+  public ServiceTimelinePublisher getServiceTimelinePublisher() {
+    return serviceTimelinePublisher;
+  }
+
+  public Map<ContainerId, ComponentInstance> getLiveInstances() {
+    return liveInstances;
+  }
+
+  public ContainerLaunchService getContainerLaunchService() {
+    return containerLaunchService;
+  }
+
+  public ServiceContext getContext() {
+    return context;
+  }
+
+  public Map<String, Component> getAllComponents() {
+    return componentsByName;
+  }
+
+  public Service getApp() {
+    return app;
+  }
+
+  public AsyncDispatcher getDispatcher() {
+    return dispatcher;
+  }
+
+  public BoundedAppender getDiagnostics() {
+    return diagnostics;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/ServiceApiConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/ServiceApiConstants.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/ServiceApiConstants.java
new file mode 100644
index 0000000..0bfb220
--- /dev/null
+++ 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/ServiceApiConstants.java
@@ -0,0 +1,69 @@
+/*
+ * 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.api;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import static org.apache.hadoop.yarn.service.utils.ServiceApiUtil.$;
+
+/**
+ * This class defines constants that can be used in input spec for
+ * variable substitutions
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public interface ServiceApiConstants {
+
+  // Constants for service
+  String SERVICE_NAME = $("SERVICE_NAME");
+
+  String SERVICE_NAME_LC = $("SERVICE_NAME.lc");
+
+  String USER = $("USER");
+
+  String DOMAIN = $("DOMAIN");
+
+  // Constants for component
+  String COMPONENT_NAME = $("COMPONENT_NAME");
+
+  String COMPONENT_NAME_LC = $("COMPONENT_NAME.lc");
+
+  String COMPONENT_INSTANCE_NAME = $("COMPONENT_INSTANCE_NAME");
+
+  // Constants for component instance
+  String COMPONENT_ID = $("COMPONENT_ID");
+
+  String CONTAINER_ID = $("CONTAINER_ID");
+
+  // Constants for default cluster ZK
+  String CLUSTER_ZK_QUORUM = $("CLUSTER_ZK_QUORUM");
+
+  // URI for the default cluster fs
+  String CLUSTER_FS_URI = $("CLUSTER_FS_URI");
+
+  // the host component of the cluster fs UI
+  String CLUSTER_FS_HOST = $("CLUSTER_FS_HOST");
+
+  // Path in zookeeper for a specific service
+  String SERVICE_ZK_PATH = $("SERVICE_ZK_PATH");
+
+  // Constants for service specific hdfs dir
+  String SERVICE_HDFS_DIR = $("SERVICE_HDFS_DIR");
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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/Artifact.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Artifact.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/Artifact.java
new file mode 100644
index 0000000..4328c6f
--- /dev/null
+++ 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/Artifact.java
@@ -0,0 +1,160 @@
+/*
+ * 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.api.records;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonValue;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Artifact of an service component.
+ **/
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+@ApiModel(description = "Artifact of an service component")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class Artifact implements Serializable {
+  private static final long serialVersionUID = 3608929500111099035L;
+
+  private String id = null;
+
+  public enum TypeEnum {
+    DOCKER("DOCKER"), TARBALL("TARBALL"), SERVICE("SERVICE");
+
+    private String value;
+
+    TypeEnum(String value) {
+      this.value = value;
+    }
+
+    @Override
+    @JsonValue
+    public String toString() {
+      return value;
+    }
+  }
+
+  private TypeEnum type = TypeEnum.DOCKER;
+  private String uri = null;
+
+  /**
+   * Artifact id. Examples are package location uri for tarball based services,
+   * image name for docker, etc.
+   **/
+  public Artifact id(String id) {
+    this.id = id;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", required = true, value = "Artifact id. Examples are package location uri for tarball based services, image name for docker, etc.")
+  @JsonProperty("id")
+  public String getId() {
+    return id;
+  }
+
+  public void setId(String id) {
+    this.id = id;
+  }
+
+  /**
+   * Artifact type, like docker, tarball, etc. (optional).
+   **/
+  public Artifact type(TypeEnum type) {
+    this.type = type;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Artifact type, like docker, tarball, etc. (optional).")
+  @JsonProperty("type")
+  public TypeEnum getType() {
+    return type;
+  }
+
+  public void setType(TypeEnum type) {
+    this.type = type;
+  }
+
+  /**
+   * Artifact location to support multiple artifact stores (optional).
+   **/
+  public Artifact uri(String uri) {
+    this.uri = uri;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Artifact location to support multiple artifact stores (optional).")
+  @JsonProperty("uri")
+  public String getUri() {
+    return uri;
+  }
+
+  public void setUri(String uri) {
+    this.uri = uri;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Artifact artifact = (Artifact) o;
+    return Objects.equals(this.id, artifact.id)
+        && Objects.equals(this.type, artifact.type)
+        && Objects.equals(this.uri, artifact.uri);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(id, type, uri);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Artifact {\n");
+
+    sb.append("    id: ").append(toIndentedString(id)).append("\n");
+    sb.append("    type: ").append(toIndentedString(type)).append("\n");
+    sb.append("    uri: ").append(toIndentedString(uri)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..b0a8e82
--- /dev/null
+++ 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
@@ -0,0 +1,412 @@
+/*
+ * 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.api.records;
+
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * One or more components of the service. If the service is HBase say,
+ * then the component can be a simple role like master or regionserver. If the
+ * service is a complex business webapp then a component can be other
+ * services say Kafka or Storm. Thereby it opens up the support for complex
+ * and nested services.
+ **/
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+@ApiModel(description = "One or more components of the service. If the service is HBase say, then the component can be a simple role like master or regionserver. If the service is a complex business webapp then a component can be other services say Kafka or Storm. Thereby it opens up the support for complex and nested services.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@XmlRootElement
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class Component implements Serializable {
+  private static final long serialVersionUID = -8430058381509087805L;
+
+  private String name = null;
+  private List<String> dependencies = new ArrayList<String>();
+  private ReadinessCheck readinessCheck = null;
+  private Artifact artifact = null;
+  private String launchCommand = null;
+  private Resource resource = null;
+  private Long numberOfContainers = null;
+  private Boolean runPrivilegedContainer = false;
+  private PlacementPolicy placementPolicy = null;
+  private Configuration configuration = new Configuration();
+  private List<String> quicklinks = new ArrayList<String>();
+  private List<Container> containers =
+      Collections.synchronizedList(new ArrayList<Container>());
+
+  /**
+   * Name of the service component (mandatory).
+   **/
+  public Component name(String name) {
+    this.name = name;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", required = true, value = "Name of the service component (mandatory).")
+  @JsonProperty("name")
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  /**
+   * An array of service components which should be in READY state (as
+   * defined by readiness check), before this component can be started. The
+   * dependencies across all components of a service should be represented
+   * as a DAG.
+   **/
+  public Component dependencies(List<String> dependencies) {
+    this.dependencies = dependencies;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "An array of service components which should be in READY state (as defined by readiness check), before this component can be started. The dependencies across all components of an service should be represented as a DAG.")
+  @JsonProperty("dependencies")
+  public List<String> getDependencies() {
+    return dependencies;
+  }
+
+  public void setDependencies(List<String> dependencies) {
+    this.dependencies = dependencies;
+  }
+
+  /**
+   * Readiness check for this component.
+   **/
+  public Component readinessCheck(ReadinessCheck readinessCheck) {
+    this.readinessCheck = readinessCheck;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Readiness check for this component.")
+  @JsonProperty("readiness_check")
+  public ReadinessCheck getReadinessCheck() {
+    return readinessCheck;
+  }
+
+  @XmlElement(name = "readiness_check")
+  public void setReadinessCheck(ReadinessCheck readinessCheck) {
+    this.readinessCheck = readinessCheck;
+  }
+
+  /**
+   * Artifact of the component (optional). If not specified, the service
+   * level global artifact takes effect.
+   **/
+  public Component artifact(Artifact artifact) {
+    this.artifact = artifact;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Artifact of the component (optional). If not specified, the service level global artifact takes effect.")
+  @JsonProperty("artifact")
+  public Artifact getArtifact() {
+    return artifact;
+  }
+
+  public void setArtifact(Artifact artifact) {
+    this.artifact = artifact;
+  }
+
+  /**
+   * The custom launch command of this component (optional). When specified at
+   * the component level, it overrides the value specified at the global level
+   * (if any).
+   **/
+  public Component launchCommand(String launchCommand) {
+    this.launchCommand = launchCommand;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "The custom launch command of this component (optional). When specified at the component level, it overrides the value specified at the global level (if any).")
+  @JsonProperty("launch_command")
+  public String getLaunchCommand() {
+    return launchCommand;
+  }
+
+  @XmlElement(name = "launch_command")
+  public void setLaunchCommand(String launchCommand) {
+    this.launchCommand = launchCommand;
+  }
+
+  /**
+   * Resource of this component (optional). If not specified, the service
+   * level global resource takes effect.
+   **/
+  public Component resource(Resource resource) {
+    this.resource = resource;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Resource of this component (optional). If not specified, the service level global resource takes effect.")
+  @JsonProperty("resource")
+  public Resource getResource() {
+    return resource;
+  }
+
+  public void setResource(Resource resource) {
+    this.resource = resource;
+  }
+
+  /**
+   * Number of containers for this component (optional). If not specified,
+   * the service level global number_of_containers takes effect.
+   **/
+  public Component numberOfContainers(Long numberOfContainers) {
+    this.numberOfContainers = numberOfContainers;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Number of containers for this component (optional). If not specified, the service level global number_of_containers takes effect.")
+  @JsonProperty("number_of_containers")
+  public Long getNumberOfContainers() {
+    return numberOfContainers;
+  }
+
+  @XmlElement(name = "number_of_containers")
+  public void setNumberOfContainers(Long numberOfContainers) {
+    this.numberOfContainers = numberOfContainers;
+  }
+
+  @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.")
+  @JsonProperty("containers")
+  public List<Container> getContainers() {
+    return containers;
+  }
+
+  public void setContainers(List<Container> containers) {
+    this.containers = containers;
+  }
+
+  public void addContainer(Container container) {
+    this.containers.add(container);
+  }
+
+  public void removeContainer(Container container) {
+    containers.remove(container);
+  }
+  public Container getContainer(String id) {
+    for (Container container : containers) {
+      if (container.getId().equals(id)) {
+        return container;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Run all containers of this component in privileged mode (YARN-4262).
+   **/
+  public Component runPrivilegedContainer(Boolean runPrivilegedContainer) {
+    this.runPrivilegedContainer = runPrivilegedContainer;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Run all containers of this component in privileged mode (YARN-4262).")
+  @JsonProperty("run_privileged_container")
+  public Boolean getRunPrivilegedContainer() {
+    return runPrivilegedContainer;
+  }
+
+  @XmlElement(name = "run_privileged_container")
+  public void setRunPrivilegedContainer(Boolean runPrivilegedContainer) {
+    this.runPrivilegedContainer = runPrivilegedContainer;
+  }
+
+  /**
+   * Advanced scheduling and placement policies for all containers of this
+   * component (optional). If not specified, the service level placement_policy
+   * takes effect. Refer to the description at the global level for more
+   * details.
+   **/
+  public Component placementPolicy(PlacementPolicy placementPolicy) {
+    this.placementPolicy = placementPolicy;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Advanced scheduling and placement policies for all containers of this component (optional). If not specified, the service level placement_policy takes effect. Refer to the description at the global level for more details.")
+  @JsonProperty("placement_policy")
+  public PlacementPolicy getPlacementPolicy() {
+    return placementPolicy;
+  }
+
+  @XmlElement(name = "placement_policy")
+  public void setPlacementPolicy(PlacementPolicy placementPolicy) {
+    this.placementPolicy = placementPolicy;
+  }
+
+  /**
+   * Config properties for this component.
+   **/
+  public Component configuration(Configuration configuration) {
+    this.configuration = configuration;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Config properties for this component.")
+  @JsonProperty("configuration")
+  public Configuration getConfiguration() {
+    return configuration;
+  }
+
+  public void setConfiguration(Configuration configuration) {
+    this.configuration = configuration;
+  }
+
+  /**
+   * A list of quicklink keys defined at the service level, and to be
+   * resolved by this component.
+   **/
+  public Component quicklinks(List<String> quicklinks) {
+    this.quicklinks = quicklinks;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A list of quicklink keys defined at the service level, and to be resolved by this component.")
+  @JsonProperty("quicklinks")
+  public List<String> getQuicklinks() {
+    return quicklinks;
+  }
+
+  public void setQuicklinks(List<String> quicklinks) {
+    this.quicklinks = quicklinks;
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Component component = (Component) o;
+    return Objects.equals(this.name, component.name)
+        && Objects.equals(this.dependencies, component.dependencies)
+        && Objects.equals(this.readinessCheck, component.readinessCheck)
+        && Objects.equals(this.artifact, component.artifact)
+        && Objects.equals(this.launchCommand, component.launchCommand)
+        && Objects.equals(this.resource, component.resource)
+        && Objects.equals(this.numberOfContainers, component.numberOfContainers)
+        && Objects.equals(this.runPrivilegedContainer,
+            component.runPrivilegedContainer)
+        && Objects.equals(this.placementPolicy, component.placementPolicy)
+        && Objects.equals(this.configuration, component.configuration)
+        && Objects.equals(this.quicklinks, component.quicklinks);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(name, dependencies, readinessCheck, artifact,
+        launchCommand, resource, numberOfContainers,
+        runPrivilegedContainer, placementPolicy, configuration, quicklinks);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Component {\n");
+
+    sb.append("    name: ").append(toIndentedString(name)).append("\n");
+    sb.append("    dependencies: ").append(toIndentedString(dependencies))
+        .append("\n");
+    sb.append("    readinessCheck: ").append(toIndentedString(readinessCheck))
+        .append("\n");
+    sb.append("    artifact: ").append(toIndentedString(artifact)).append("\n");
+    sb.append("    launchCommand: ").append(toIndentedString(launchCommand))
+        .append("\n");
+    sb.append("    resource: ").append(toIndentedString(resource)).append("\n");
+    sb.append("    numberOfContainers: ")
+        .append(toIndentedString(numberOfContainers)).append("\n");
+    sb.append("    containers: ").append(toIndentedString(containers))
+        .append("\n");
+    sb.append("    runPrivilegedContainer: ")
+        .append(toIndentedString(runPrivilegedContainer)).append("\n");
+    sb.append("    placementPolicy: ").append(toIndentedString(placementPolicy))
+        .append("\n");
+    sb.append("    configuration: ").append(toIndentedString(configuration))
+        .append("\n");
+    sb.append("    quicklinks: ").append(toIndentedString(quicklinks))
+        .append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+
+  /**
+   * Merge from another component into this component without overwriting.
+   */
+  public void mergeFrom(Component that) {
+    if (this.getArtifact() == null) {
+      this.setArtifact(that.getArtifact());
+    }
+    if (this.getResource() == null) {
+      this.setResource(that.getResource());
+    }
+    if (this.getNumberOfContainers() == null) {
+      this.setNumberOfContainers(that.getNumberOfContainers());
+    }
+    if (this.getLaunchCommand() == null) {
+      this.setLaunchCommand(that.getLaunchCommand());
+    }
+    this.getConfiguration().mergeFrom(that.getConfiguration());
+    if (this.getQuicklinks() == null) {
+      this.setQuicklinks(that.getQuicklinks());
+    }
+    if (this.getRunPrivilegedContainer() == null) {
+      this.setRunPrivilegedContainer(that.getRunPrivilegedContainer());
+    }
+    if (this.getDependencies() == null) {
+      this.setDependencies(that.getDependencies());
+    }
+    if (this.getPlacementPolicy() == null) {
+      this.setPlacementPolicy(that.getPlacementPolicy());
+    }
+    if (this.getReadinessCheck() == null) {
+      this.setReadinessCheck(that.getReadinessCheck());
+    }
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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/ConfigFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ConfigFormat.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/ConfigFormat.java
new file mode 100644
index 0000000..e10305a
--- /dev/null
+++ 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/ConfigFormat.java
@@ -0,0 +1,67 @@
+/*
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+import java.util.Locale;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public enum ConfigFormat {
+
+  JSON("json"),
+  PROPERTIES("properties"),
+  XML("xml"),
+  HADOOP_XML("hadoop_xml"),
+  ENV("env"),
+  TEMPLATE("template"),
+  YAML("yaml"),
+  ;
+  ConfigFormat(String suffix) {
+    this.suffix = suffix;
+  }
+
+  private final String suffix;
+
+  public String getSuffix() {
+    return suffix;
+  }
+
+
+  @Override
+  public String toString() {
+    return suffix;
+  }
+
+  /**
+   * Get a matching format or null
+   * @param type
+   * @return the format
+   */
+  public static ConfigFormat resolve(String type) {
+    for (ConfigFormat format: values()) {
+      if (format.getSuffix().equals(type.toLowerCase(Locale.ENGLISH))) {
+        return format;
+      }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/Configuration.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/Configuration.java
new file mode 100644
index 0000000..2f8ca96
--- /dev/null
+++ 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/Configuration.java
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.service.api.records;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.swagger.annotations.ApiModel;
+import io.swagger.annotations.ApiModelProperty;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.yarn.service.utils.SliderUtils;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Set of configuration properties that can be injected into the service
+ * components via envs, files and custom pluggable helper docker containers.
+ * Files of several standard formats like xml, properties, json, yaml and
+ * templates will be supported.
+ **/
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+@ApiModel(description = "Set of configuration properties that can be injected into the service components via envs, files and custom pluggable helper docker containers. Files of several standard formats like xml, properties, json, yaml and templates will be supported.")
+@javax.annotation.Generated(value = "class io.swagger.codegen.languages.JavaClientCodegen", date = "2016-06-02T08:15:05.615-07:00")
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class Configuration implements Serializable {
+  private static final long serialVersionUID = -4330788704981074466L;
+
+  private Map<String, String> properties = new HashMap<String, String>();
+  private Map<String, String> env = new HashMap<String, String>();
+  private List<ConfigFile> files = new ArrayList<ConfigFile>();
+
+  /**
+   * A blob of key-value pairs of common service properties.
+   **/
+  public Configuration properties(Map<String, String> properties) {
+    this.properties = properties;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A blob of key-value pairs of common service properties.")
+  @JsonProperty("properties")
+  public Map<String, String> getProperties() {
+    return properties;
+  }
+
+  public void setProperties(Map<String, String> properties) {
+    this.properties = properties;
+  }
+
+  /**
+   * A blob of key-value pairs which will be appended to the default system
+   * properties and handed off to the service at start time. All placeholder
+   * references to properties will be substituted before injection.
+   **/
+  public Configuration env(Map<String, String> env) {
+    this.env = env;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "A blob of key-value pairs which will be appended to the default system properties and handed off to the service at start time. All placeholder references to properties will be substituted before injection.")
+  @JsonProperty("env")
+  public Map<String, String> getEnv() {
+    return env;
+  }
+
+  public void setEnv(Map<String, String> env) {
+    this.env = env;
+  }
+
+  /**
+   * Array of list of files that needs to be created and made available as
+   * volumes in the service component containers.
+   **/
+  public Configuration files(List<ConfigFile> files) {
+    this.files = files;
+    return this;
+  }
+
+  @ApiModelProperty(example = "null", value = "Array of list of files that needs to be created and made available as volumes in the service component containers.")
+  @JsonProperty("files")
+  public List<ConfigFile> getFiles() {
+    return files;
+  }
+
+  public void setFiles(List<ConfigFile> files) {
+    this.files = files;
+  }
+
+  public long getPropertyLong(String name, long defaultValue) {
+    String value = getProperty(name);
+    if (StringUtils.isEmpty(value)) {
+      return defaultValue;
+    }
+    return Long.parseLong(value);
+  }
+
+  public int getPropertyInt(String name, int defaultValue) {
+    String value = getProperty(name);
+    if (StringUtils.isEmpty(value)) {
+      return defaultValue;
+    }
+    return Integer.parseInt(value);
+  }
+
+  public boolean getPropertyBool(String name, boolean defaultValue) {
+    String value = getProperty(name);
+    if (StringUtils.isEmpty(value)) {
+      return defaultValue;
+    }
+    return Boolean.parseBoolean(value);
+  }
+
+  public String getProperty(String name, String defaultValue) {
+    String value = getProperty(name);
+    if (StringUtils.isEmpty(value)) {
+      return defaultValue;
+    }
+    return value;
+  }
+
+  public void setProperty(String name, String value) {
+    properties.put(name, value);
+  }
+
+  public String getProperty(String name) {
+    return properties.get(name.trim());
+  }
+
+  public String getEnv(String name) {
+    return env.get(name.trim());
+  }
+
+  @Override
+  public boolean equals(java.lang.Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    Configuration configuration = (Configuration) o;
+    return Objects.equals(this.properties, configuration.properties)
+        && Objects.equals(this.env, configuration.env)
+        && Objects.equals(this.files, configuration.files);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(properties, env, files);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("class Configuration {\n");
+
+    sb.append("    properties: ").append(toIndentedString(properties))
+        .append("\n");
+    sb.append("    env: ").append(toIndentedString(env)).append("\n");
+    sb.append("    files: ").append(toIndentedString(files)).append("\n");
+    sb.append("}");
+    return sb.toString();
+  }
+
+  /**
+   * Convert the given object to string with each line indented by 4 spaces
+   * (except the first line).
+   */
+  private String toIndentedString(java.lang.Object o) {
+    if (o == null) {
+      return "null";
+    }
+    return o.toString().replace("\n", "\n    ");
+  }
+
+  /**
+   * Merge all properties and envs from that configuration to this configration.
+   * For ConfigFiles, all properties and envs of that ConfigFile are merged into
+   * this ConfigFile.
+   */
+  public synchronized void mergeFrom(Configuration that) {
+    SliderUtils.mergeMapsIgnoreDuplicateKeys(this.properties, that
+        .getProperties());
+    SliderUtils.mergeMapsIgnoreDuplicateKeys(this.env, that.getEnv());
+
+    Map<String, ConfigFile> thatMap = new HashMap<>();
+    for (ConfigFile file : that.getFiles()) {
+      thatMap.put(file.getDestFile(), file.copy());
+    }
+    for (ConfigFile thisFile : files) {
+      if(thatMap.containsKey(thisFile.getDestFile())) {
+        ConfigFile thatFile = thatMap.get(thisFile.getDestFile());
+        SliderUtils.mergeMapsIgnoreDuplicateKeys(thisFile.getProps(),
+            thatFile.getProps());
+        thatMap.remove(thisFile.getDestFile());
+      }
+    }
+    // add remaining new files from that Configration
+    for (ConfigFile thatFile : thatMap.values()) {
+      files.add(thatFile.copy());
+    }
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4f8fe178/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/ContainerState.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/api/records/ContainerState.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/ContainerState.java
new file mode 100644
index 0000000..bf09ff2
--- /dev/null
+++ 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/ContainerState.java
@@ -0,0 +1,30 @@
+/*
+ * 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.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * The current state of the container of an application.
+ **/
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public enum ContainerState {
+  RUNNING_BUT_UNREADY, READY, STOPPED
+}


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