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/06/21 18:33:31 UTC

[09/50] [abbrv] hadoop git commit: YARN-6255. Refactor yarn-native-services framework. Contributed by Jian He

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerClientProvider.java
index 86d87ac..8b88c28 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerClientProvider.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerClientProvider.java
@@ -22,7 +22,6 @@ import org.apache.slider.common.SliderKeys;
 import org.apache.slider.common.tools.SliderFileSystem;
 import org.apache.slider.core.conf.AggregateConf;
 import org.apache.slider.core.conf.ConfTreeOperations;
-import org.apache.slider.core.exceptions.BadConfigException;
 import org.apache.slider.core.exceptions.SliderException;
 import org.apache.slider.providers.AbstractClientProvider;
 import org.apache.slider.providers.ProviderRole;
@@ -30,14 +29,10 @@ import org.apache.slider.providers.ProviderUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
-import static org.apache.slider.providers.docker.DockerKeys.DOCKER_IMAGE;
-
 public class DockerClientProvider extends AbstractClientProvider
     implements SliderKeys {
 
@@ -64,35 +59,7 @@ public class DockerClientProvider extends AbstractClientProvider
   public void validateInstanceDefinition(AggregateConf instanceDefinition,
       SliderFileSystem fs) throws SliderException {
     super.validateInstanceDefinition(instanceDefinition, fs);
-
-    ConfTreeOperations appConf = instanceDefinition.getAppConfOperations();
-    ConfTreeOperations resources = instanceDefinition.getResourceOperations();
-
-    for (String roleGroup : resources.getComponentNames()) {
-      if (roleGroup.equals(COMPONENT_AM)) {
-        continue;
-      }
-      if (appConf.getComponentOpt(roleGroup, DOCKER_IMAGE, null) == null &&
-          appConf.getGlobalOptions().get(DOCKER_IMAGE) == null) {
-        throw new BadConfigException("Property " + DOCKER_IMAGE + " not " +
-            "specified for " + roleGroup);
-      }
-
-      providerUtils.getPackages(roleGroup, appConf);
-
-      if (appConf.getComponentOptBool(roleGroup, AM_CONFIG_GENERATION, false)) {
-        // build and localize configuration files
-        Map<String, Map<String, String>> configurations =
-            providerUtils.buildConfigurations(appConf, appConf, null,
-                null, roleGroup, roleGroup, null);
-        try {
-          providerUtils.localizeConfigFiles(null, roleGroup, roleGroup, appConf,
-              configurations, null, fs, null);
-        } catch (IOException e) {
-          throw new BadConfigException(e.toString());
-        }
-      }
-    }
+    //TODO validate Application payload, part of that is already done in ApplicationApiService, need to do more
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
index 63416cc..511f7bc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/docker/DockerProviderService.java
@@ -17,286 +17,129 @@
  */
 package org.apache.slider.providers.docker;
 
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 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.LocalResource;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
-import org.apache.slider.api.ClusterDescription;
-import org.apache.slider.api.ClusterNode;
-import org.apache.slider.api.OptionKeys;
+import org.apache.slider.api.resource.Application;
+import org.apache.slider.api.resource.Component;
+import org.apache.slider.api.resource.ContainerState;
 import org.apache.slider.common.SliderKeys;
 import org.apache.slider.common.tools.SliderFileSystem;
 import org.apache.slider.common.tools.SliderUtils;
-import org.apache.slider.core.conf.AggregateConf;
-import org.apache.slider.core.conf.ConfTreeOperations;
-import org.apache.slider.core.conf.MapOperations;
 import org.apache.slider.core.exceptions.SliderException;
 import org.apache.slider.core.launch.CommandLineBuilder;
 import org.apache.slider.core.launch.ContainerLauncher;
-import org.apache.slider.core.registry.docstore.ConfigFormat;
-import org.apache.slider.core.registry.docstore.ConfigUtils;
-import org.apache.slider.core.registry.docstore.ExportEntry;
-import org.apache.slider.providers.AbstractProviderService;
-import org.apache.slider.providers.MonitorDetail;
-import org.apache.slider.providers.ProviderCore;
+import org.apache.slider.core.registry.docstore.PublishedConfiguration;
 import org.apache.slider.providers.ProviderRole;
+import org.apache.slider.providers.ProviderService;
 import org.apache.slider.providers.ProviderUtils;
 import org.apache.slider.server.appmaster.state.RoleInstance;
+import org.apache.slider.server.appmaster.state.StateAccessForProviders;
+import org.apache.slider.server.services.yarnregistry.YarnRegistryViewForProviders;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.File;
 import java.io.IOException;
-import java.net.URL;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Scanner;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.regex.Pattern;
 
-public class DockerProviderService extends AbstractProviderService implements
-    ProviderCore,
-    DockerKeys,
-    SliderKeys {
+public class DockerProviderService extends AbstractService
+    implements ProviderService, DockerKeys, SliderKeys {
 
   protected static final Logger log =
       LoggerFactory.getLogger(DockerProviderService.class);
   private static final ProviderUtils providerUtils = new ProviderUtils(log);
-  private static final String EXPORT_GROUP = "quicklinks";
-  private static final String APPLICATION_TAG = "application";
-  private static final String HOST_KEY_FORMAT = "${%s_HOST}";
-  private static final String IP_KEY_FORMAT = "${%s_IP}";
-  private static final String VARIABLE_INDICATOR = "${";
-
-  private String clusterName = null;
-  private SliderFileSystem fileSystem = null;
-
-  private final Map<String, Set<ExportEntry>> exportMap =
-      new ConcurrentHashMap<>();
+  private static final String QUICK_LINKS = "quicklinks";
+  protected StateAccessForProviders amState;
+  protected YarnRegistryViewForProviders yarnRegistry;
 
   protected DockerProviderService() {
     super("DockerProviderService");
   }
 
   @Override
-  public List<ProviderRole> getRoles() {
-    return Collections.emptyList();
-  }
-
-  @Override
-  public boolean isSupportedRole(String role) {
-    return true;
+  public void setAMState(StateAccessForProviders stateAccessor) {
+    this.amState = stateAccessor;
   }
 
   @Override
-  public void validateInstanceDefinition(AggregateConf instanceDefinition)
-      throws SliderException {
+  public void bindToYarnRegistry(YarnRegistryViewForProviders yarnRegistry) {
+    this.yarnRegistry = yarnRegistry;
   }
 
-  private String getClusterName() {
-    if (SliderUtils.isUnset(clusterName)) {
-      clusterName = getAmState().getInternalsSnapshot().get(OptionKeys.APPLICATION_NAME);
-    }
-    return clusterName;
-  }
-
-  @Override
   public void buildContainerLaunchContext(ContainerLauncher launcher,
-      AggregateConf instanceDefinition, Container container,
-      ProviderRole providerRole, SliderFileSystem fileSystem,
-      Path generatedConfPath, MapOperations resourceComponent,
-      MapOperations appComponent, Path containerTmpDirPath)
+      Application application, Container container, ProviderRole providerRole,
+      SliderFileSystem fileSystem)
       throws IOException, SliderException {
 
     String roleName = providerRole.name;
     String roleGroup = providerRole.group;
 
-    log.info("Build launch context for Docker");
-    log.debug(instanceDefinition.toString());
-
-    ConfTreeOperations appConf = instanceDefinition.getAppConfOperations();
+    Component component = providerRole.component;
     launcher.setYarnDockerMode(true);
-    launcher.setDockerImage(appConf.getComponentOpt(roleGroup, DOCKER_IMAGE,
-        null));
-    launcher.setDockerNetwork(appConf.getComponentOpt(roleGroup, DOCKER_NETWORK,
-        DEFAULT_DOCKER_NETWORK));
-    launcher.setRunPrivilegedContainer(appConf.getComponentOptBool(roleGroup,
-        DOCKER_USE_PRIVILEGED, DEFAULT_DOCKER_USE_PRIVILEGED));
-
-    // Set the environment
-    Map<String, String> standardTokens = providerUtils.getStandardTokenMap(
-        getAmState().getAppConfSnapshot(), getAmState().getInternalsSnapshot(),
-        roleName, roleGroup, container.getId().toString(), getClusterName());
-    Map<String, String> replaceTokens = providerUtils.filterSiteOptions(
-            appConf.getComponent(roleGroup).options, standardTokens);
-    replaceTokens.putAll(standardTokens);
-    launcher.putEnv(SliderUtils.buildEnvMap(appComponent, replaceTokens));
-
-    String workDir = ApplicationConstants.Environment.PWD.$();
-    launcher.setEnv("WORK_DIR", workDir);
-    log.info("WORK_DIR set to {}", workDir);
-    String logDir = ApplicationConstants.LOG_DIR_EXPANSION_VAR;
-    launcher.setEnv("LOG_DIR", logDir);
-    log.info("LOG_DIR set to {}", logDir);
+    launcher.setDockerImage(component.getArtifact().getId());
+    launcher.setDockerNetwork(component.getConfiguration()
+        .getProperty(DOCKER_NETWORK, DEFAULT_DOCKER_NETWORK));
+    launcher.setRunPrivilegedContainer(component.getRunPrivilegedContainer());
+
+    // Generate tokens (key-value pair) for config substitution.
+    Map<String, String> standardTokens = providerUtils
+        .getStandardTokenMap(application.getConfiguration(),
+            component.getConfiguration(), roleName, roleGroup,
+            container.getId().toString(), application.getName());
+    Map<String, String> tokensForSubstitution = providerUtils.substituteConfigs(
+            component.getConfiguration().getProperties(), standardTokens);
+
+    tokensForSubstitution.putAll(standardTokens);
+
+    // Set the environment variables
+    launcher.putEnv(SliderUtils
+        .buildEnvMap(component.getConfiguration(), tokensForSubstitution));
+    launcher.setEnv("WORK_DIR", ApplicationConstants.Environment.PWD.$());
+    launcher.setEnv("LOG_DIR", ApplicationConstants.LOG_DIR_EXPANSION_VAR);
     if (System.getenv(HADOOP_USER_NAME) != null) {
       launcher.setEnv(HADOOP_USER_NAME, System.getenv(HADOOP_USER_NAME));
     }
-    //add english env
     launcher.setEnv("LANG", "en_US.UTF-8");
     launcher.setEnv("LC_ALL", "en_US.UTF-8");
     launcher.setEnv("LANGUAGE", "en_US.UTF-8");
 
-    //local resources
-    providerUtils.localizePackages(launcher, fileSystem, appConf, roleGroup,
-        getClusterName());
-
-    if (SliderUtils.isHadoopClusterSecure(getConfig())) {
-      providerUtils.localizeServiceKeytabs(launcher, instanceDefinition,
-          fileSystem, getClusterName());
+    for (Entry<String, String> entry : launcher.getEnv().entrySet()) {
+      tokensForSubstitution.put("${" + entry.getKey() + "}", entry.getValue());
     }
 
-    if (appComponent.getOptionBool(AM_CONFIG_GENERATION, false)) {
-      // build and localize configuration files
-      Map<String, Map<String, String>> configurations =
-          providerUtils.buildConfigurations(
-              instanceDefinition.getAppConfOperations(),
-              instanceDefinition.getInternalOperations(),
-              container.getId().toString(), getClusterName(),
-              roleName, roleGroup, getAmState());
-      providerUtils.localizeConfigFiles(launcher, roleName, roleGroup,
-          appConf, configurations, launcher.getEnv(), fileSystem,
-          getClusterName());
+    providerUtils.addRoleHostTokens(tokensForSubstitution, amState);
+
+    log.info("Token for substitution: " + tokensForSubstitution);
+
+    if (SliderUtils.isHadoopClusterSecure(getConfig())) {
+      //TODO localize key tabs, WHY is this code needed ? WHY DOES CONTAINER REQUIRE AM KEYTAB??
+      providerUtils.localizeServiceKeytabs(launcher, fileSystem, application);
     }
 
-    //add the configuration resources
-    launcher.addLocalResources(fileSystem.submitDirectory(
-        generatedConfPath,
-        PROPAGATED_CONF_DIR_NAME));
+    // create config file on hdfs and add local resource
+    providerUtils.createConfigFileAndAddLocalResource(launcher, fileSystem,
+        component, tokensForSubstitution, amState);
 
     CommandLineBuilder operation = new CommandLineBuilder();
-    operation.add(appConf.getComponentOpt(roleGroup, DOCKER_START_COMMAND,
-        "/bin/bash"));
-
+    operation.add(component.getLaunchCommand());
     operation.add("> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/"
         + OUT_FILE + " 2>" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/"
         + ERR_FILE);
-
     launcher.addCommand(operation.build());
 
-    // Additional files to localize
-    String appResourcesString = instanceDefinition.getAppConfOperations()
-        .getGlobalOptions().getOption(APP_RESOURCES, null);
-    log.info("Configuration value for extra resources to localize: {}", appResourcesString);
-    if (null != appResourcesString) {
-      try (Scanner scanner = new Scanner(appResourcesString).useDelimiter(",")) {
-        while (scanner.hasNext()) {
-          String resource = scanner.next();
-          Path resourcePath = new Path(resource);
-          LocalResource extraResource = fileSystem.createAmResource(
-              fileSystem.getFileSystem().resolvePath(resourcePath),
-              LocalResourceType.FILE);
-          String destination = APP_RESOURCES_DIR + "/" + resourcePath.getName();
-          log.info("Localizing {} to {}", resourcePath, destination);
-          // TODO Can we try harder to avoid collisions?
-          launcher.addLocalResource(destination, extraResource);
-        }
-      }
-    }
+    // publish exports
+    // TODO move this to app level, no need to do this for every container launch
+    providerUtils
+        .substituteConfigs(application.getQuicklinks(), tokensForSubstitution);
+    PublishedConfiguration pubconf = new PublishedConfiguration(QUICK_LINKS,
+        application.getQuicklinks().entrySet());
+    amState.getPublishedSliderConfigurations().put(QUICK_LINKS, pubconf);
   }
 
-  @Override
-  public void initializeApplicationConfiguration(
-      AggregateConf instanceDefinition, SliderFileSystem fileSystem,
-      String roleGroup)
-      throws IOException, SliderException {
-        this.fileSystem = fileSystem;
-  }
-
-  @Override
-  public void applyInitialRegistryDefinitions(URL amWebURI,
-      ServiceRecord serviceRecord)
-      throws IOException {
-    super.applyInitialRegistryDefinitions(amWebURI, serviceRecord);
-
-    // identify client component
-    String clientName = null;
-    ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
-    for (String component : appConf.getComponentNames()) {
-      if (COMPONENT_TYPE_CLIENT.equals(appConf.getComponentOpt(component,
-          COMPONENT_TYPE_KEY, null))) {
-        clientName = component;
-        break;
-      }
-    }
-    if (clientName == null) {
-      log.info("No client component specified, not publishing client configs");
-      return;
-    }
-
-    // register AM-generated client configs
-    // appConf should already be resolved!
-    MapOperations clientOperations = appConf.getComponent(clientName);
-    if (!clientOperations.getOptionBool(AM_CONFIG_GENERATION, false)) {
-      log.info("AM config generation is false, not publishing client configs");
-      return;
-    }
-
-    // build and localize configuration files
-    Map<String, Map<String, String>> configurations =
-        providerUtils.buildConfigurations(appConf, getAmState()
-            .getInternalsSnapshot(), null, getClusterName(), clientName,
-            clientName, getAmState());
-
-    for (Map.Entry<String, Map<String, String>>  entry : configurations.entrySet()) {
-      String configFileDN = entry.getKey();
-      String configFileName = appConf.getComponentOpt(clientName,
-          OptionKeys.CONF_FILE_PREFIX + configFileDN + OptionKeys
-              .NAME_SUFFIX, null);
-      String configFileType = appConf.getComponentOpt(clientName,
-          OptionKeys.CONF_FILE_PREFIX + configFileDN + OptionKeys
-              .TYPE_SUFFIX, null);
-      if (configFileName == null || configFileType == null) {
-        continue;
-      }
-      ConfigFormat configFormat = ConfigFormat.resolve(configFileType);
-
-      Map<String, String> config = entry.getValue();
-      ConfigUtils.prepConfigForTemplateOutputter(configFormat, config,
-          fileSystem, getClusterName(),
-          new File(configFileName).getName());
-      providerUtils.publishApplicationInstanceData(configFileDN, configFileDN,
-          config.entrySet(), getAmState());
-    }
-  }
-
-  @Override
-  public void notifyContainerCompleted(ContainerId containerId) {
-    if (containerId != null) {
-      String containerIdStr = containerId.toString();
-      log.info("Removing container exports for {}", containerIdStr);
-      for (Set<ExportEntry> exportEntries : exportMap.values()) {
-        for (Iterator<ExportEntry> iter = exportEntries.iterator();
-            iter.hasNext();) {
-          ExportEntry entry = iter.next();
-          if (containerIdStr.equals(entry.getContainerId())) {
-            iter.remove();
-          }
-        }
-      }
-    }
-  }
-
-  @Override
   public boolean processContainerStatus(ContainerId containerId,
       ContainerStatus status) {
     log.debug("Handling container status: {}", status);
@@ -304,144 +147,24 @@ public class DockerProviderService extends AbstractProviderService implements
         SliderUtils.isUnset(status.getHost())) {
       return true;
     }
-    RoleInstance instance = getAmState().getOwnedContainer(containerId);
+    RoleInstance instance = amState.getOwnedContainer(containerId);
     if (instance == null) {
       // container is completed?
       return false;
     }
 
-    String roleName = instance.role;
-    String roleGroup = instance.group;
-    String containerIdStr = containerId.toString();
-
-    providerUtils.updateServiceRecord(getAmState(), yarnRegistry,
-        containerIdStr, roleName, status.getIPs(), status.getHost());
-
-    publishExportGroups(containerIdStr, roleName, roleGroup,
-        status.getHost(), status.getIPs());
-    return false;
-  }
-
-  /**
-   * This method looks for configuration properties of the form
-   * export.key,value and publishes the key,value pair. Standard tokens are
-   * substituted into the value, and COMPONENTNAME_HOST and THIS_HOST tokens
-   * are substituted with the actual hostnames of the containers.
-   */
-  protected void publishExportGroups(String containerId,
-      String roleName, String roleGroup, String thisHost, List<String> ips) {
-    ConfTreeOperations appConf = getAmState().getAppConfSnapshot();
-    ConfTreeOperations internalsConf = getAmState().getInternalsSnapshot();
-
-    Map<String, String> exports = providerUtils.getExports(
-        getAmState().getAppConfSnapshot(), roleGroup);
-
-    // publish export groups if any
-    Map<String, String> standardTokens = providerUtils.getStandardTokenMap(
-        appConf, internalsConf, roleName, roleGroup, containerId,
-        getClusterName());
-    Map<String, String> replaceTokens = providerUtils.filterSiteOptions(
-            appConf.getComponent(roleGroup).options, standardTokens);
-    replaceTokens.putAll(standardTokens);
-
-    String roleNameKey = providerUtils.getNameKey(roleName, roleGroup,
-        appConf);
-    String roleNameIPKey = null;
-    if (roleNameKey != null) {
-      replaceTokens.put(String.format(HOST_KEY_FORMAT, roleNameKey), thisHost);
-      roleNameIPKey = Pattern.quote(String.format(IP_KEY_FORMAT, roleNameKey));
-    } else {
-      // should not happen, but log if it does
-      log.info("Not replacing HOST or IP tokens because key was null for {}",
-          roleName);
-    }
-    String roleGroupKey = providerUtils.getGroupKey(roleGroup, appConf);
-    String roleGroupIPKey = null;
-    if (roleGroupKey != null) {
-      if (roleNameKey == null || !roleGroupKey.equals(roleNameKey)) {
-        replaceTokens.put(String.format(HOST_KEY_FORMAT, roleGroupKey),
-            thisHost);
-        roleGroupIPKey = Pattern.quote(String.format(IP_KEY_FORMAT,
-            roleGroupKey));
-      }
+    providerUtils.updateServiceRecord(amState, yarnRegistry,
+        containerId.toString(), instance.role, status.getIPs(), status.getHost());
+    // TODO publish ip and host
+    org.apache.slider.api.resource.Container container =
+        instance.providerRole.component.getContainer(containerId.toString());
+    if (container != null) {
+      container.setIp(StringUtils.join(",", status.getIPs()));
+      container.setHostname(status.getHost());
+      container.setState(ContainerState.READY);
     } else {
-      // should not happen, but log if it does
-      log.info("Not replacing HOST or IP tokens because key was null for {}",
-          roleGroup);
-    }
-    replaceTokens.put("${THIS_HOST}", thisHost);
-
-    for (Entry<String, String> export : exports.entrySet()) {
-      String value = export.getValue();
-      // replace host names and site properties
-      for (Map.Entry<String, String>  entry : replaceTokens.entrySet()) {
-        String token = entry.getKey();
-        if (value.contains(token)) {
-          value = value.replaceAll(Pattern.quote(token), entry.getValue());
-        }
-      }
-      Set<String> values = new HashSet<>();
-      for (String ip : ips) {
-        values.add(substituteIP(roleNameIPKey, roleGroupIPKey, ip, value));
-      }
-      for (String exportValue : values) {
-        if (exportValue.contains(VARIABLE_INDICATOR)) {
-          // not all variables have been substituted, so do not export
-          continue;
-        }
-        ExportEntry entry = new ExportEntry();
-        entry.setContainerId(containerId);
-        entry.setLevel(APPLICATION_TAG);
-        entry.setValue(exportValue);
-        entry.setUpdatedTime(new Date().toString());
-        Set<ExportEntry> exportEntries = getExportEntries(export.getKey());
-        exportEntries.add(entry);
-        log.info("Preparing to publish for {}. Key {} and Value {}",
-            roleName, export.getKey(), entry);
-      }
-    }
-    if (!exportMap.isEmpty()) {
-      providerUtils.publishExportGroup(exportMap, getAmState(), EXPORT_GROUP);
-    }
-  }
-
-  protected String substituteIP(String roleNameIPKey, String roleGroupIPKey,
-      String ip, String value) {
-    if (roleNameIPKey != null) {
-      value = value.replaceAll(roleNameIPKey, ip);
-    }
-    if (roleGroupIPKey != null) {
-      value = value.replaceAll(roleGroupIPKey, ip);
-    }
-    return value;
-  }
-
-  protected Set<ExportEntry> getExportEntries(String key) {
-    if (!this.exportMap.containsKey(key)) {
-      synchronized (this.exportMap) {
-        if (!this.exportMap.containsKey(key)) {
-          this.exportMap.put(key, Collections.newSetFromMap(
-              new ConcurrentHashMap<>()));
-        }
-      }
-    }
-
-    return this.exportMap.get(key);
-  }
-
-  @Override
-  public Map<String, MonitorDetail> buildMonitorDetails(ClusterDescription clusterDesc) {
-    Map<String, MonitorDetail> details = super.buildMonitorDetails(clusterDesc);
-    buildRoleHostDetails(details);
-    return details;
-  }
-
-  private void buildRoleHostDetails(Map<String, MonitorDetail> details) {
-    for (Map.Entry<String, Map<String, ClusterNode>> entry :
-        getAmState().getRoleClusterNodeMapping().entrySet()) {
-      details.put(entry.getKey() + " Host(s)/Container(s)",
-          new MonitorDetail(providerUtils.getHostsList(
-              entry.getValue().values(), false).toString(), false));
+      log.warn(containerId + " not found in Application!");
     }
+    return false;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
deleted file mode 100644
index b58d3aa..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
+++ /dev/null
@@ -1,305 +0,0 @@
-/*
- * 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.slider.providers.slideram;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.yarn.api.records.LocalResource;
-import org.apache.hadoop.yarn.api.records.LocalResourceType;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.slider.api.InternalKeys;
-import org.apache.slider.api.ResourceKeys;
-import org.apache.slider.api.RoleKeys;
-import org.apache.slider.common.SliderKeys;
-import org.apache.slider.common.SliderXmlConfKeys;
-import org.apache.slider.common.tools.SliderFileSystem;
-import org.apache.slider.common.tools.SliderUtils;
-import org.apache.slider.core.conf.AggregateConf;
-import org.apache.slider.core.conf.MapOperations;
-import org.apache.slider.core.exceptions.BadClusterStateException;
-import org.apache.slider.core.exceptions.BadConfigException;
-import org.apache.slider.core.exceptions.SliderException;
-import org.apache.slider.core.launch.AbstractLauncher;
-import org.apache.slider.core.launch.JavaCommandLineBuilder;
-import org.apache.slider.providers.AbstractClientProvider;
-import org.apache.slider.providers.PlacementPolicy;
-import org.apache.slider.providers.ProviderRole;
-import org.apache.slider.providers.ProviderUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.slider.api.ResourceKeys.COMPONENT_INSTANCES;
-
-/**
- * handles the setup of the Slider AM.
- * This keeps aspects of role, cluster validation and Clusterspec setup
- * out of the core slider client
- */
-public class SliderAMClientProvider extends AbstractClientProvider
-    implements SliderKeys {
-
-
-  protected static final Logger log =
-    LoggerFactory.getLogger(SliderAMClientProvider.class);
-  protected static final String NAME = "SliderAM";
-  public static final String INSTANCE_RESOURCE_BASE = PROVIDER_RESOURCE_BASE_ROOT +
-                                                       "slideram/instance/";
-  public static final String INTERNAL_JSON =
-    INSTANCE_RESOURCE_BASE + "internal.json";
-  public static final String APPCONF_JSON =
-    INSTANCE_RESOURCE_BASE + "appconf.json";
-  public static final String RESOURCES_JSON =
-    INSTANCE_RESOURCE_BASE + "resources.json";
-
-  public SliderAMClientProvider(Configuration conf) {
-    super(conf);
-  }
-
-  /**
-   * List of roles
-   */
-  public static final List<ProviderRole> ROLES =
-    new ArrayList<ProviderRole>();
-
-  public static final int KEY_AM = ROLE_AM_PRIORITY_INDEX;
-
-  public static final ProviderRole APPMASTER =
-      new ProviderRole(COMPONENT_AM, KEY_AM,
-          PlacementPolicy.EXCLUDE_FROM_FLEXING,
-          ResourceKeys.DEFAULT_NODE_FAILURE_THRESHOLD, 
-          0, "");
-
-  /**
-   * Initialize role list
-   */
-  static {
-    ROLES.add(APPMASTER);
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-
-  @Override
-  public List<ProviderRole> getRoles() {
-    return ROLES;
-  }
-
-
-  @Override //Client
-  public void preflightValidateClusterConfiguration(SliderFileSystem sliderFileSystem,
-                                                    String clustername,
-                                                    Configuration configuration,
-                                                    AggregateConf instanceDefinition,
-                                                    Path clusterDirPath,
-                                                    Path generatedConfDirPath,
-                                                    boolean secure)
-      throws SliderException, IOException {
-
-    super.preflightValidateClusterConfiguration(sliderFileSystem, clustername, configuration, instanceDefinition, clusterDirPath, generatedConfDirPath, secure);
-    //add a check for the directory being writeable by the current user
-    String
-      dataPath = instanceDefinition.getInternalOperations()
-                                   .getGlobalOptions()
-                                   .getMandatoryOption(
-                                     InternalKeys.INTERNAL_DATA_DIR_PATH);
-
-    Path path = new Path(dataPath);
-    sliderFileSystem.verifyDirectoryWriteAccess(path);
-    Path historyPath = new Path(clusterDirPath, SliderKeys.HISTORY_DIR_NAME);
-    sliderFileSystem.verifyDirectoryWriteAccess(historyPath);
-  }
-
-  /**
-   * Verify that an instance definition is considered valid by the provider
-   * @param instanceDefinition instance definition
-   * @throws SliderException if the configuration is not valid
-   */
-  public void validateInstanceDefinition(AggregateConf instanceDefinition, SliderFileSystem fs) throws
-      SliderException {
-
-    super.validateInstanceDefinition(instanceDefinition, fs);
-    
-    // make sure there is no negative entry in the instance count
-    Map<String, Map<String, String>> instanceMap =
-        instanceDefinition.getResources().components;
-    for (Map.Entry<String, Map<String, String>> entry : instanceMap.entrySet()) {
-      MapOperations mapOperations = new MapOperations(entry);
-      int instances = mapOperations.getOptionInt(COMPONENT_INSTANCES, 0);
-      if (instances < 0) {
-        throw new BadClusterStateException(
-            "Component %s has negative instance count: %d",
-            mapOperations.name,
-            instances);
-      }
-    }
-  }
-  
-  /**
-   * The Slider AM sets up all the dependency JARs above slider.jar itself
-   * {@inheritDoc}
-   */
-  public void prepareAMAndConfigForLaunch(SliderFileSystem fileSystem,
-      Configuration serviceConf,
-      AbstractLauncher launcher,
-      AggregateConf instanceDescription,
-      Path snapshotConfDirPath,
-      Path generatedConfDirPath,
-      Configuration clientConfExtras,
-      String libdir,
-      Path tempPath, boolean miniClusterTestRun)
-    throws IOException, SliderException {
-
-    Map<String, LocalResource> providerResources = new HashMap<>();
-
-    ProviderUtils.addProviderJar(providerResources,
-        this,
-        SLIDER_JAR,
-        fileSystem,
-        tempPath,
-        libdir,
-        miniClusterTestRun);
-
-    log.info("Loading all dependencies for AM.");
-    // If slider.tar.gz is available in hdfs use it, else upload all jars
-    Path dependencyLibTarGzip = fileSystem.getDependencyTarGzip();
-    if (fileSystem.isFile(dependencyLibTarGzip)) {
-      SliderUtils.putAmTarGzipAndUpdate(providerResources, fileSystem);
-    } else {
-      for (String libDirProp : SliderUtils.getLibDirs()) {
-        ProviderUtils.addAllDependencyJars(providerResources,
-                                           fileSystem,
-                                           tempPath,
-                                           libdir,
-                                           libDirProp);
-
-      }
-    }
-    addKeytabResourceIfNecessary(fileSystem,
-                                 instanceDescription,
-                                 providerResources);
-
-    launcher.addLocalResources(providerResources);
-
-    //also pick up all env variables from a map
-    launcher.copyEnvVars(
-      instanceDescription.getInternalOperations().getOrAddComponent(
-        SliderKeys.COMPONENT_AM));
-  }
-
-  /**
-   * If the cluster is secure, and an HDFS installed keytab is available for AM
-   * authentication, add this keytab as a local resource for the AM launch.
-   *
-   * @param fileSystem
-   * @param instanceDescription
-   * @param providerResources
-   * @throws IOException
-   * @throws BadConfigException if there's no keytab and it is explicitly required.
-   */
-  protected void addKeytabResourceIfNecessary(SliderFileSystem fileSystem,
-                                              AggregateConf instanceDescription,
-                                              Map<String, LocalResource> providerResources)
-    throws IOException, BadConfigException {
-    if (UserGroupInformation.isSecurityEnabled()) {
-      String keytabPathOnHost = instanceDescription.getAppConfOperations()
-          .getComponent(SliderKeys.COMPONENT_AM).get(
-              SliderXmlConfKeys.KEY_AM_KEYTAB_LOCAL_PATH);
-      if (SliderUtils.isUnset(keytabPathOnHost)) {
-        String amKeytabName = instanceDescription.getAppConfOperations()
-            .getComponent(SliderKeys.COMPONENT_AM).get(
-                SliderXmlConfKeys.KEY_AM_LOGIN_KEYTAB_NAME);
-        String keytabDir = instanceDescription.getAppConfOperations()
-            .getComponent(SliderKeys.COMPONENT_AM).get(
-                SliderXmlConfKeys.KEY_HDFS_KEYTAB_DIR);
-        Path keytabPath = fileSystem.buildKeytabPath(keytabDir, amKeytabName,
-                                                     instanceDescription.getName());
-        if (fileSystem.getFileSystem().exists(keytabPath)) {
-          LocalResource keytabRes = fileSystem.createAmResource(keytabPath,
-                                                  LocalResourceType.FILE);
-
-          providerResources.put(SliderKeys.KEYTAB_DIR + "/" +
-                                 amKeytabName, keytabRes);
-        } else {
-          log.warn("No keytab file was found at {}.", keytabPath);
-          if (getConf().getBoolean(KEY_AM_LOGIN_KEYTAB_REQUIRED, false)) {
-            throw new BadConfigException("No keytab file was found at %s.", keytabPath);
-
-          } else {
-            log.warn("The AM will be "
-              + "started without a kerberos authenticated identity. "
-              + "The application is therefore not guaranteed to remain "
-              + "operational beyond 24 hours.");
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   * Update the AM resource with any local needs
-   * @param capability capability to update
-   */
-  public void prepareAMResourceRequirements(MapOperations sliderAM,
-                                            Resource capability) {
-    capability.setMemory(sliderAM.getOptionInt(
-      ResourceKeys.YARN_MEMORY,
-      capability.getMemory()));
-    capability.setVirtualCores(
-        sliderAM.getOptionInt(ResourceKeys.YARN_CORES, capability.getVirtualCores()));
-  }
-  
-  /**
-   * Extract any JVM options from the cluster specification and
-   * add them to the command line
-   */
-  public void addJVMOptions(AggregateConf aggregateConf,
-                            JavaCommandLineBuilder cmdLine)
-      throws BadConfigException {
-
-    MapOperations sliderAM =
-        aggregateConf.getAppConfOperations().getMandatoryComponent(
-        SliderKeys.COMPONENT_AM);
-    cmdLine.forceIPv4().headless();
-    String heap = sliderAM.getOption(RoleKeys.JVM_HEAP,
-                                   DEFAULT_JVM_HEAP);
-    cmdLine.setJVMHeap(heap);
-    String jvmopts = sliderAM.getOption(RoleKeys.JVM_OPTS, "");
-    if (SliderUtils.isSet(jvmopts)) {
-      cmdLine.add(jvmopts);
-    }
-  }
-
-
-  @Override
-  public void prepareInstanceConfiguration(AggregateConf aggregateConf)
-      throws SliderException, IOException {
-    mergeTemplates(aggregateConf,
-        INTERNAL_JSON, RESOURCES_JSON, APPCONF_JSON
-                  );
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
deleted file mode 100644
index c021b80..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
+++ /dev/null
@@ -1,185 +0,0 @@
-/*
- * 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.slider.providers.slideram;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.yarn.api.records.Container;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.registry.client.binding.RegistryTypeUtils;
-import org.apache.hadoop.registry.client.types.ServiceRecord;
-import org.apache.slider.common.SliderKeys;
-import org.apache.slider.common.tools.ConfigHelper;
-import org.apache.slider.common.tools.SliderFileSystem;
-import org.apache.slider.common.tools.SliderUtils;
-import org.apache.slider.core.conf.AggregateConf;
-import org.apache.slider.core.conf.MapOperations;
-import org.apache.slider.core.exceptions.BadCommandArgumentsException;
-import org.apache.slider.core.exceptions.SliderException;
-import org.apache.slider.core.launch.ContainerLauncher;
-import org.apache.slider.core.registry.docstore.PublishedConfiguration;
-import org.apache.slider.core.registry.info.CustomRegistryConstants;
-import org.apache.slider.providers.AbstractProviderService;
-import org.apache.slider.providers.ProviderCore;
-import org.apache.slider.providers.ProviderRole;
-import org.apache.slider.providers.agent.AgentKeys;
-import org.apache.slider.server.appmaster.PublishedArtifacts;
-import org.apache.slider.server.appmaster.web.rest.RestPaths;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.apache.slider.server.appmaster.web.rest.RestPaths.*;
-
-/**
- * Exists just to move some functionality out of AppMaster into a peer class
- * of the actual service provider doing the real work
- */
-public class SliderAMProviderService extends AbstractProviderService implements
-    ProviderCore,
-    AgentKeys,
-    SliderKeys {
-
-  public SliderAMProviderService() {
-    super("SliderAMProviderService");
-  }
-
-  @Override
-  public String getHumanName() {
-    return "Slider Application";
-  }
-  
-  @Override
-  public Configuration loadProviderConfigurationInformation(File confDir) throws
-      BadCommandArgumentsException,
-      IOException {
-    return null;
-  }
-
-  @Override
-  public void buildContainerLaunchContext(ContainerLauncher containerLauncher,
-      AggregateConf instanceDefinition,
-      Container container,
-      ProviderRole role,
-      SliderFileSystem sliderFileSystem,
-      Path generatedConfPath,
-      MapOperations resourceComponent,
-      MapOperations appComponent,
-      Path containerTmpDirPath) throws IOException, SliderException {
-  }
-
-  @Override
-  public List<ProviderRole> getRoles() {
-    return new ArrayList<>(0);
-  }
-
-  @Override
-  public void validateInstanceDefinition(AggregateConf instanceDefinition) throws
-      SliderException {
-
-  }
-
-  @Override
-  public void applyInitialRegistryDefinitions(URL amWebURI,
-      ServiceRecord serviceRecord)
-      throws IOException {
-    super.applyInitialRegistryDefinitions(amWebURI,
-        serviceRecord);
-    // now publish site.xml files
-    YarnConfiguration defaultYarnConfig = new YarnConfiguration();
-    amState.getPublishedSliderConfigurations().put(
-        PublishedArtifacts.COMPLETE_CONFIG,
-        new PublishedConfiguration(
-            "Complete slider application settings",
-            getConfig(), getConfig()));
-    amState.getPublishedSliderConfigurations().put(
-        PublishedArtifacts.YARN_SITE_CONFIG,
-        new PublishedConfiguration(
-            "YARN site settings",
-            ConfigHelper.loadFromResource("yarn-site.xml"),
-            defaultYarnConfig) );
-
-    amState.getPublishedSliderConfigurations().put(
-        PublishedArtifacts.CORE_SITE_CONFIG,
-        new PublishedConfiguration(
-            "Core site settings",
-            ConfigHelper.loadFromResource("core-site.xml"),
-            defaultYarnConfig) );
-    amState.getPublishedSliderConfigurations().put(
-        PublishedArtifacts.HDFS_SITE_CONFIG,
-        new PublishedConfiguration(
-            "HDFS site settings",
-            ConfigHelper.loadFromResource("hdfs-site.xml"),
-            new HdfsConfiguration(true)) );
-
-
-    try {
-
-      URL managementAPI = new URL(amWebURI, RELATIVE_PATH_MANAGEMENT);
-      URL registryREST = new URL(amWebURI, RELATIVE_PATH_REGISTRY);
-
-      URL publisherURL = new URL(amWebURI, RELATIVE_PATH_PUBLISHER);
-
-      // Set the configurations URL.
-
-      String configurationsURL = SliderUtils.appendToURL(
-          publisherURL.toExternalForm(), RestPaths.SLIDER_CONFIGSET);
-      String exportsURL = SliderUtils.appendToURL(
-          publisherURL.toExternalForm(), RestPaths.SLIDER_EXPORTS);
-
-      serviceRecord.addExternalEndpoint(
-          RegistryTypeUtils.webEndpoint(
-              CustomRegistryConstants.WEB_UI, amWebURI.toURI()));
-      
-      serviceRecord.addExternalEndpoint(
-          RegistryTypeUtils.webEndpoint(
-              CustomRegistryConstants.AM_REST_BASE, amWebURI.toURI()));
-      
-      serviceRecord.addExternalEndpoint(
-          RegistryTypeUtils.restEndpoint(
-              CustomRegistryConstants.MANAGEMENT_REST_API,
-              managementAPI.toURI()));
-      serviceRecord.addExternalEndpoint(
-          RegistryTypeUtils.restEndpoint(
-              CustomRegistryConstants.PUBLISHER_REST_API,
-              publisherURL.toURI()));
-      serviceRecord.addExternalEndpoint(
-          RegistryTypeUtils.restEndpoint(
-              CustomRegistryConstants.REGISTRY_REST_API,
-              registryREST.toURI()));
-      serviceRecord.addExternalEndpoint(
-          RegistryTypeUtils.restEndpoint(
-              CustomRegistryConstants.PUBLISHER_CONFIGURATIONS_API,
-              new URI(configurationsURL)));
-      serviceRecord.addExternalEndpoint(
-          RegistryTypeUtils.restEndpoint(
-              CustomRegistryConstants.PUBLISHER_EXPORTS_API,
-              new URI(exportsURL)));
-
-    } catch (URISyntaxException e) {
-      throw new IOException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68ec5e78/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/RoleLaunchService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/RoleLaunchService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/RoleLaunchService.java
index 3cfe167..70eab71 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/RoleLaunchService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-slider/hadoop-yarn-slider-core/src/main/java/org/apache/slider/server/appmaster/RoleLaunchService.java
@@ -18,16 +18,12 @@
 
 package org.apache.slider.server.appmaster;
 
-import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.slider.common.SliderKeys;
+import org.apache.slider.api.resource.Application;
 import org.apache.slider.common.tools.SliderFileSystem;
-import org.apache.slider.core.conf.AggregateConf;
-import org.apache.slider.core.conf.MapOperations;
 import org.apache.slider.core.launch.ContainerLauncher;
 import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.providers.ProviderService;
@@ -37,8 +33,8 @@ import org.apache.slider.server.appmaster.actions.QueueAccess;
 import org.apache.slider.server.appmaster.state.ContainerAssignment;
 import org.apache.slider.server.appmaster.state.RoleInstance;
 import org.apache.slider.server.appmaster.state.RoleStatus;
-import org.apache.slider.server.services.workflow.WorkflowExecutorService;
 import org.apache.slider.server.services.workflow.ServiceThreadFactory;
+import org.apache.slider.server.services.workflow.WorkflowExecutorService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -73,16 +69,6 @@ public class RoleLaunchService
    */
   private final SliderFileSystem fs;
 
-  /**
-   * Path in the launch filesystem that refers to a configuration directory
-   * -the interpretation of it is left to the Provider
-   */
-  private final Path generatedConfDirPath;
-  /**
-   * Path in the launch filesystem that refers to a temp directory
-   * which will be cleaned up at (some) time in the future
-   */
-  private final Path launcherTmpDirPath;
 
   private Map<String, String> envVars;
 
@@ -91,21 +77,13 @@ public class RoleLaunchService
    * @param queueAccess
    * @param provider the provider
    * @param fs filesystem
-   * @param generatedConfDirPath path in the FS for the generated dir
    * @param envVars environment variables
-   * @param launcherTmpDirPath path for a temporary data in the launch process
    */
-  public RoleLaunchService(QueueAccess queueAccess,
-      ProviderService provider,
-      SliderFileSystem fs,
-      Path generatedConfDirPath,
-      Map<String, String> envVars,
-      Path launcherTmpDirPath) {
+  public RoleLaunchService(QueueAccess queueAccess, ProviderService provider,
+      SliderFileSystem fs, Map<String, String> envVars) {
     super(ROLE_LAUNCH_SERVICE);
     this.actionQueue = queueAccess;
     this.fs = fs;
-    this.generatedConfDirPath = generatedConfDirPath;
-    this.launcherTmpDirPath = launcherTmpDirPath;
     this.provider = provider;
     this.envVars = envVars;
   }
@@ -120,23 +98,13 @@ public class RoleLaunchService
   /**
    * Start an asychronous launch operation
    * @param assignment container assignment
-   * @param clusterSpec cluster spec to use for template
    * @param credentials credentials to use
    */
   public void launchRole(ContainerAssignment assignment,
-      AggregateConf clusterSpec,
-      Credentials credentials) {
-    RoleStatus role = assignment.role;
-    String roleName = role.getName();
-    String roleGroup = role.getGroup();
-    // prelaunch safety check
-    Preconditions.checkArgument(provider.isSupportedRole(roleName));
+      Application application, Credentials credentials) {
     RoleLaunchService.RoleLauncher launcher =
-      new RoleLaunchService.RoleLauncher(assignment,
-         clusterSpec,
-         clusterSpec.getResourceOperations().getOrAddComponent(roleGroup),
-         clusterSpec.getAppConfOperations().getOrAddComponent(roleGroup),
-         credentials);
+        new RoleLaunchService.RoleLauncher(assignment, application,
+            credentials);
     execute(launcher);
   }
 
@@ -148,35 +116,21 @@ public class RoleLaunchService
     private final ContainerAssignment assignment;
     // Allocated container
     public final Container container;
-    private final MapOperations resourceComponent;
-    private final MapOperations appComponent;
-    private final AggregateConf instanceDefinition;
+    public final Application application;
     public final ProviderRole role;
     private final Credentials credentials;
-    private Exception raisedException;
 
     public RoleLauncher(ContainerAssignment assignment,
-        AggregateConf instanceDefinition,
-        MapOperations resourceComponent,
-        MapOperations appComponent,
+        Application application,
         Credentials credentials) {
       this.assignment = assignment;
       this.credentials = credentials;
       this.container = assignment.container;
       RoleStatus roleStatus = assignment.role;
-
-      assert resourceComponent != null;
-      assert appComponent != null;
       ProviderRole providerRole = roleStatus.getProviderRole();
-      assert providerRole != null;
       this.role = providerRole;
-      this.resourceComponent = resourceComponent;
-      this.appComponent = appComponent;
-      this.instanceDefinition = instanceDefinition;
-    }
+      this.application = application;
 
-    public Exception getRaisedException() {
-      return raisedException;
     }
 
     @Override
@@ -196,62 +150,46 @@ public class RoleLaunchService
         containerLauncher.setupUGI();
         containerLauncher.putEnv(envVars);
 
-        log.debug("Launching container {} into role {}",
-                  container.getId(),
-                  role.name);
+        log.info("Launching container {} into RoleName = {}, RoleGroup = {}",
+            container.getId(), role.name, role.group);
 
-        //now build up the configuration data
-        Path containerTmpDirPath =
-          new Path(launcherTmpDirPath, container.getId().toString());
-        provider.buildContainerLaunchContext(containerLauncher,
-            instanceDefinition,
-            container,
-            role,
-            fs,
-            generatedConfDirPath,
-            resourceComponent,
-            appComponent,
-            containerTmpDirPath);
+        provider.buildContainerLaunchContext(containerLauncher, application,
+            container, role, fs);
 
         RoleInstance instance = new RoleInstance(container);
         String[] envDescription = containerLauncher.dumpEnvToString();
 
         String commandsAsString = containerLauncher.getCommandsAsString();
-        log.info("Starting container with command: {}",
-                 commandsAsString);
+        log.info("Starting container with command: {}", commandsAsString);
 
+        instance.providerRole = role;
         instance.command = commandsAsString;
         instance.role = role.name;
         instance.group = role.group;
         instance.roleId = role.id;
-        instance.appVersion = instanceDefinition.getAppConfOperations()
-            .getGlobalOptions().get(SliderKeys.APP_VERSION);
         instance.environment = envDescription;
-        int delay = appComponent.getOptionInt(
-            AgentKeys.KEY_CONTAINER_LAUNCH_DELAY, 0);
-        int maxDelay =
-            getConfig().getInt(YarnConfiguration.RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS,
-                               YarnConfiguration.DEFAULT_RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS);
+        long delay = role.component.getConfiguration()
+            .getPropertyLong(AgentKeys.KEY_CONTAINER_LAUNCH_DELAY, 0);
+        long maxDelay = getConfig()
+            .getLong(YarnConfiguration.RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS,
+                YarnConfiguration.DEFAULT_RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS);
         if (delay > maxDelay/1000) {
           log.warn("Container launch delay of {} exceeds the maximum allowed of"
                    + " {} seconds.  Delay will not be utilized.",
                    delay, maxDelay/1000);
           delay = 0;
         }
-        log.info("Container launch delay for {} set to {} seconds",
-                 role.name, delay);
-        actionQueue.schedule(new ActionStartContainer("starting " + role.name,
-                                                      container,
-                                                      containerLauncher.completeContainerLaunch(),
-                                                      instance,
-                                                      delay,
-                                                      TimeUnit.SECONDS));
+        log.info("Container launch delay for {} set to {} seconds", role.name,
+            delay);
+        actionQueue.schedule(
+            new ActionStartContainer("starting " + role.name, container,
+                containerLauncher.completeContainerLaunch(), instance, delay,
+                TimeUnit.SECONDS));
       } catch (Exception e) {
-        log.error("Exception thrown while trying to start {}: {}",
-            role.name, e, e);
-        raisedException = e;
+        log.error("Exception thrown while trying to start " + role.name
+            + " container = " + container.getId() + " on host " + container
+            .getNodeId(), e);
       }
     }
-
   }
 }


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