You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by bi...@apache.org on 2016/05/13 01:55:05 UTC

[4/7] incubator-slider git commit: SLIDER-1107 add client install capability in the case of AM generated configs

SLIDER-1107 add client install capability in the case of AM generated configs


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/bafc5e72
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/bafc5e72
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/bafc5e72

Branch: refs/heads/feature/SLIDER-1107_AM_config_generation
Commit: bafc5e726e0bfdb5adb08e1755bc8ad5cd9f3ec2
Parents: 3c47ec2
Author: Billie Rinaldi <bi...@gmail.com>
Authored: Tue May 10 10:09:57 2016 -0700
Committer: Billie Rinaldi <bi...@gmail.com>
Committed: Thu May 12 18:51:37 2016 -0700

----------------------------------------------------------------------
 .../org/apache/slider/client/ClientUtils.java   | 111 ++++++++++++++++
 .../org/apache/slider/client/SliderClient.java  |  91 ++++---------
 .../slider/common/tools/CoreFileSystem.java     |  13 ++
 .../slider/core/conf/ConfTreeOperations.java    |  16 ++-
 .../providers/AbstractClientProvider.java       |   9 +-
 .../providers/agent/AgentClientProvider.java    | 127 ++++++++++++++++---
 .../slider/providers/agent/AgentKeys.java       |   3 +
 .../providers/agent/AgentProviderService.java   |   4 +-
 .../agent/TestAgentClientProvider2.java         |   8 +-
 9 files changed, 294 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/bafc5e72/slider-core/src/main/java/org/apache/slider/client/ClientUtils.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/ClientUtils.java b/slider-core/src/main/java/org/apache/slider/client/ClientUtils.java
new file mode 100644
index 0000000..c3ccb1d
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/client/ClientUtils.java
@@ -0,0 +1,111 @@
+/*
+ * 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.client;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.PathNotFoundException;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
+import org.apache.hadoop.registry.client.binding.RegistryPathUtils;
+import org.apache.hadoop.registry.client.exceptions.NoRecordException;
+import org.apache.hadoop.registry.client.types.ServiceRecord;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.core.exceptions.BadCommandArgumentsException;
+import org.apache.slider.core.exceptions.NotFoundException;
+import org.apache.slider.core.exceptions.SliderException;
+import org.apache.slider.core.registry.docstore.ConfigFormat;
+import org.apache.slider.core.registry.docstore.PublishedConfigSet;
+import org.apache.slider.core.registry.docstore.PublishedConfiguration;
+import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
+import org.apache.slider.core.registry.retrieve.RegistryRetriever;
+
+import java.io.File;
+import java.io.IOException;
+
+import static org.apache.hadoop.registry.client.binding.RegistryUtils.currentUser;
+import static org.apache.hadoop.registry.client.binding.RegistryUtils.servicePath;
+
+public class ClientUtils {
+  public static ServiceRecord lookupServiceRecord(RegistryOperations rops,
+      String user, String name) throws IOException, SliderException {
+    return lookupServiceRecord(rops, user, null, name);
+  }
+
+  public static ServiceRecord lookupServiceRecord(RegistryOperations rops,
+      String user, String type, String name) throws IOException,
+      SliderException {
+    if (StringUtils.isEmpty(user)) {
+      user = currentUser();
+    } else {
+      user = RegistryPathUtils.encodeForRegistry(user);
+    }
+    if (StringUtils.isEmpty(type)) {
+      type = SliderKeys.APP_TYPE;
+    }
+
+    String path = servicePath(user, type, name);
+    return resolve(rops, path);
+  }
+
+  public static ServiceRecord resolve(RegistryOperations rops, String path)
+      throws IOException, SliderException {
+    try {
+      return rops.resolve(path);
+    } catch (PathNotFoundException | NoRecordException e) {
+      throw new NotFoundException(e.getPath().toString(), e);
+    }
+  }
+
+  public static PublishedConfiguration getConfigFromRegistry(
+      RegistryOperations rops, Configuration configuration,
+      String configName, String appName, String user, boolean external)
+      throws IOException, SliderException {
+    ServiceRecord instance = lookupServiceRecord(rops, user, appName);
+
+    RegistryRetriever retriever = new RegistryRetriever(configuration, instance);
+    PublishedConfigSet configurations = retriever.getConfigurations(external);
+
+    PublishedConfiguration published = retriever.retrieveConfiguration(
+        configurations, configName, external);
+    return published;
+  }
+
+  public static String saveOrReturnConfig(PublishedConfiguration published,
+      String format, File destPath, String fileName)
+      throws BadCommandArgumentsException, IOException {
+    ConfigFormat configFormat = ConfigFormat.resolve(format);
+    if (configFormat == null) {
+      throw new BadCommandArgumentsException(
+          "Unknown/Unsupported format %s ", format);
+    }
+    PublishedConfigurationOutputter outputter =
+        PublishedConfigurationOutputter.createOutputter(configFormat,
+            published);
+    boolean print = destPath == null;
+    if (!print) {
+      if (destPath.isDirectory()) {
+        // creating it under a directory
+        destPath = new File(destPath, fileName);
+      }
+      outputter.save(destPath);
+      return null;
+    } else {
+      return outputter.asString();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/bafc5e72/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
index f332a66..eb0630d 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClient.java
@@ -1301,8 +1301,21 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
         E_INVALID_INSTALL_PATH + ": " + clientInfo.installLocation.getAbsolutePath());
 
     File pkgFile;
-    require(isSet(clientInfo.packageURI), E_INVALID_APPLICATION_PACKAGE_LOCATION);
-    pkgFile = new File(clientInfo.packageURI);
+    File tmpDir = null;
+
+    require(isSet(clientInfo.packageURI) || isSet(clientInfo.name),
+        E_INVALID_APPLICATION_PACKAGE_LOCATION);
+    if (isSet(clientInfo.packageURI)) {
+      pkgFile = new File(clientInfo.packageURI);
+    } else {
+      Path appDirPath = sliderFileSystem.buildAppDefDirPath(clientInfo.name);
+      Path appDefPath = new Path(appDirPath, SliderKeys.DEFAULT_APP_PKG);
+      require(sliderFileSystem.isFile(appDefPath),
+          E_INVALID_APPLICATION_PACKAGE_LOCATION);
+      tmpDir = Files.createTempDir();
+      pkgFile = new File(tmpDir, SliderKeys.DEFAULT_APP_PKG);
+      sliderFileSystem.copyHdfsFileToLocal(appDefPath, pkgFile);
+    }
     require(pkgFile.isFile(),
         E_UNABLE_TO_READ_SUPPLIED_PACKAGE_FILE + " at %s", pkgFile.getAbsolutePath());
 
@@ -1324,6 +1337,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     AbstractClientProvider
         provider = createClientProvider(SliderProviderFactory.DEFAULT_CLUSTER_TYPE);
     provider.processClientOperation(sliderFileSystem,
+        getRegistryOperations(),
+        getConfig(),
         "INSTALL",
         clientInfo.installLocation,
         pkgFile,
@@ -4164,17 +4179,9 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   @VisibleForTesting
   public PublishedConfiguration actionRegistryGetConfig(ActionRegistryArgs registryArgs)
       throws YarnException, IOException {
-    ServiceRecord instance = lookupServiceRecord(registryArgs);
-
-    RegistryRetriever retriever = new RegistryRetriever(getConfig(), instance);
-    boolean external = !registryArgs.internal;
-    PublishedConfigSet configurations =
-        retriever.getConfigurations(external);
-
-    PublishedConfiguration published = retriever.retrieveConfiguration(configurations,
-            registryArgs.getConf,
-            external);
-    return published;
+    return ClientUtils.getConfigFromRegistry(getRegistryOperations(),
+        getConfig(), registryArgs.getConf, registryArgs.name, registryArgs.user,
+        !registryArgs.internal);
   }
 
   /**
@@ -4217,27 +4224,11 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     // decide whether or not to print
     String entry = registryArgs.getConf;
     String format = registryArgs.format;
-    ConfigFormat configFormat = ConfigFormat.resolve(format);
-    if (configFormat == null) {
-      throw new BadCommandArgumentsException(
-          "Unknown/Unsupported format %s ", format);
+    String output = ClientUtils.saveOrReturnConfig(published,
+        registryArgs.format, registryArgs.out, entry + "." + format);
+    if (output != null) {
+      print(output);
     }
-    PublishedConfigurationOutputter outputter =
-        PublishedConfigurationOutputter.createOutputter(configFormat,
-            published);
-    boolean print = registryArgs.out == null;
-    if (!print) {
-      File outputPath = registryArgs.out;
-      if (outputPath.isDirectory()) {
-        // creating it under a directory
-        outputPath = new File(outputPath, entry + "." + format);
-      }
-      log.debug("Destination path: {}", outputPath);
-      outputter.save(outputPath);
-    } else {
-      print(outputter.asString());
-    }
-    
   }
 
   /**
@@ -4287,32 +4278,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   private ServiceRecord lookupServiceRecord(ActionRegistryArgs registryArgs) throws
       SliderException,
       IOException {
-    String user;
-    if (StringUtils.isNotEmpty(registryArgs.user)) {
-      user = RegistryPathUtils.encodeForRegistry(registryArgs.user);
-    } else {
-      user = currentUser();
-    }
-
-    String path = servicePath(user, registryArgs.serviceType,
-        registryArgs.name);
-    return resolve(path);
-  }
-
-  /**
-   * Look up a service record of the current user
-   * @param serviceType service type
-   * @param id instance ID
-   * @return instance data
-   * @throws UnknownApplicationInstanceException no path or service record
-   * at the end of the path
-   * @throws SliderException other failures
-   * @throws IOException IO problems or wrapped exceptions
-   */
-  public ServiceRecord lookupServiceRecord(String serviceType, String id)
-      throws IOException, SliderException {
-    String path = servicePath(currentUser(), serviceType, id);
-    return resolve(path);
+    return ClientUtils.lookupServiceRecord(getRegistryOperations(),
+        registryArgs.user, registryArgs.serviceType, registryArgs.name);
   }
 
   /**
@@ -4327,11 +4294,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    */
   public ServiceRecord resolve(String path)
       throws IOException, SliderException {
-    try {
-      return getRegistryOperations().resolve(path);
-    } catch (PathNotFoundException | NoRecordException e) {
-      throw new NotFoundException(e.getPath().toString(), e);
-    }
+    return ClientUtils.resolve(getRegistryOperations(), path);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/bafc5e72/slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java b/slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java
index 0d3436b..df086d1 100644
--- a/slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java
+++ b/slider-core/src/main/java/org/apache/slider/common/tools/CoreFileSystem.java
@@ -765,6 +765,19 @@ public class CoreFileSystem {
     fileSystem.setPermission(destPath, fp);
   }
 
+  public void copyHdfsFileToLocal(Path hdfsPath, File destFile)
+      throws IOException {
+    if (hdfsPath == null || destFile == null) {
+      throw new IOException("Either hdfsPath or destPath is null");
+    }
+    log.info("Copying file {} to {}", hdfsPath.toUri(), destFile.toURI());
+
+    Path destPath = new Path(destFile.getPath());
+    fileSystem.copyToLocalFile(hdfsPath, destPath);
+    // set file permissions of the destPath
+    fileSystem.setPermission(destPath, new FsPermission("755"));
+  }
+
   /**
    * list entries in a filesystem directory
    *

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/bafc5e72/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java b/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
index 9013edb..7e88abd 100644
--- a/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
+++ b/slider-core/src/main/java/org/apache/slider/core/conf/ConfTreeOperations.java
@@ -345,7 +345,21 @@ public class ConfTreeOperations {
        confTreeSerDeser.fromFile(resource) );
     return ops;
   }
-  
+
+  /**
+   * Load from a json String. The inner conf tree is the loaded data -unresolved
+   * @param json json string
+   * @return loaded value
+   * @throws IOException load failure
+   */
+  public static ConfTreeOperations fromString(String json) throws
+      IOException {
+    ConfTreeSerDeser confTreeSerDeser = new ConfTreeSerDeser();
+    ConfTreeOperations ops = new ConfTreeOperations(
+        confTreeSerDeser.fromJson(json) );
+    return ops;
+  }
+
   /**
    * Build from an existing instance -which is cloned via JSON ser/deser
    * @param instance the source instance

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/bafc5e72/slider-core/src/main/java/org/apache/slider/providers/AbstractClientProvider.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/AbstractClientProvider.java b/slider-core/src/main/java/org/apache/slider/providers/AbstractClientProvider.java
index fcab65e..510de5d 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/AbstractClientProvider.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/AbstractClientProvider.java
@@ -21,6 +21,7 @@ package org.apache.slider.providers;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
 import org.apache.slider.common.tools.SliderFileSystem;
 import org.apache.slider.core.conf.AggregateConf;
 import org.apache.slider.core.conf.ConfTreeOperations;
@@ -223,18 +224,22 @@ public abstract class AbstractClientProvider extends Configured {
   /**
    * Process client operations for applications such as install, configure
    * @param fileSystem
+   * @param registryOperations
+   * @param configuration
    * @param operation
    * @param clientInstallPath
    * @param clientPackage
-   * @param config
+   * @param clientConfig
    * @param name
    * @throws SliderException
    */
   public void processClientOperation(SliderFileSystem fileSystem,
+                                     RegistryOperations registryOperations,
+                                     Configuration configuration,
                                      String operation,
                                      File clientInstallPath,
                                      File clientPackage,
-                                     JSONObject config,
+                                     JSONObject clientConfig,
                                      String name)
       throws SliderException {
     throw new SliderException("Provider does not support client operations.");

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/bafc5e72/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
index 12d581c..e5c3bec 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
@@ -26,9 +26,11 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.registry.client.api.RegistryOperations;
 import org.apache.hadoop.registry.client.binding.RegistryUtils;
 import org.apache.slider.api.InternalKeys;
 import org.apache.slider.api.ResourceKeys;
+import org.apache.slider.client.ClientUtils;
 import org.apache.slider.common.SliderKeys;
 import org.apache.slider.common.tools.SliderFileSystem;
 import org.apache.slider.common.tools.SliderUtils;
@@ -38,13 +40,18 @@ import org.apache.slider.core.conf.MapOperations;
 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.registry.docstore.PublishedConfiguration;
 import org.apache.slider.providers.AbstractClientProvider;
 import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.providers.ProviderUtils;
 import org.apache.slider.providers.agent.application.metadata.Application;
 import org.apache.slider.providers.agent.application.metadata.Component;
+import org.apache.slider.providers.agent.application.metadata.ConfigFile;
 import org.apache.slider.providers.agent.application.metadata.Metainfo;
 import org.apache.slider.providers.agent.application.metadata.MetainfoParser;
+import org.apache.slider.providers.agent.application.metadata.OSPackage;
+import org.apache.slider.providers.agent.application.metadata.OSSpecific;
+import org.apache.slider.providers.agent.application.metadata.Package;
 import org.codehaus.jettison.json.JSONException;
 import org.codehaus.jettison.json.JSONObject;
 import org.slf4j.Logger;
@@ -291,6 +298,8 @@ public class AgentClientProvider extends AbstractClientProvider
 
   @Override
   public void processClientOperation(SliderFileSystem fileSystem,
+                                     RegistryOperations rops,
+                                     Configuration configuration,
                                      String operation,
                                      File clientInstallPath,
                                      File appPackage,
@@ -379,38 +388,105 @@ public class AgentClientProvider extends AbstractClientProvider
         throw new BadConfigException(E_COULD_NOT_READ_METAINFO);
       }
 
-      expandAgentTar(agentPkgDir);
-
-      JSONObject commandJson = getCommandJson(defaultConfig, config, metaInfo, clientInstallPath, name);
-      FileWriter file = new FileWriter(new File(cmdDir, "command.json"));
-      try {
-        file.write(commandJson.toString());
-
-      } catch (IOException e) {
-        e.printStackTrace();
-      } finally {
-        file.flush();
-        file.close();
-      }
-
       String client_script = null;
+      Component clientComponent = null;
       for (Component component : metaInfo.getApplication().getComponents()) {
         if (component.getCategory().equals("CLIENT")) {
+          clientComponent = component;
           client_script = component.getCommandScript().getScript();
-          log.info("Installing CLIENT {} using script {}", component.getName(), client_script);
           break;
         }
       }
 
       if (SliderUtils.isUnset(client_script)) {
-        throw new SliderException("No valid CLIENT component found. Aborting install.");
-      }
+        log.info("Installing CLIENT without script");
+        List<Package> packages = metaInfo.getApplication().getPackages();
+        if (packages != null && packages.size() > 0) {
+          // retrieve package resources from HDFS and extract
+          for (Package pkg : packages) {
+            Path pkgPath = fileSystem.buildResourcePath(pkg.getName());
+            if (!fileSystem.isFile(pkgPath) && name != null) {
+              pkgPath = fileSystem.buildResourcePath(name, pkg.getName());
+            }
+            if (!fileSystem.isFile(pkgPath)) {
+              throw new IOException("Package doesn't exist as a resource: " +
+                  pkg.getName());
+            }
+            if ("archive".equals(pkg.getType())) {
+              File pkgFile = new File(tmpDir, pkg.getName());
+              fileSystem.copyHdfsFileToLocal(pkgPath, pkgFile);
+              expandTar(pkgFile, clientInstallPath);
+            } else {
+              File pkgFile = new File(clientInstallPath, pkg.getName());
+              fileSystem.copyHdfsFileToLocal(pkgPath, pkgFile);
+            }
+          }
+        } else {
+          // extract tarball from app def
+          for (OSSpecific osSpecific : metaInfo.getApplication()
+              .getOSSpecifics()) {
+            for (OSPackage pkg : osSpecific.getPackages()) {
+              if ("tarball".equals(pkg.getType())) {
+                File pkgFile = new File(appPkgDir, pkg.getName());
+                expandTar(pkgFile, clientInstallPath);
+              }
+            }
+          }
+        }
+        if (name == null) {
+          log.warn("Conf files not being generated because no app name was " +
+              "providied");
+          return;
+        }
+        File confInstallDir;
+        String clientRoot = null;
+        try {
+          clientRoot = defaultConfig.getJSONObject("global")
+              .getString(AgentKeys.APP_CLIENT_ROOT);
+        } catch (JSONException e) {
+          e.printStackTrace();
+        }
+        if (clientRoot == null) {
+          confInstallDir = clientInstallPath;
+        } else {
+          confInstallDir = new File(new File(clientInstallPath, clientRoot), "conf");
+          if (!confInstallDir.exists()) {
+            confInstallDir.mkdirs();
+          }
+        }
+        String user = RegistryUtils.currentUser();
+        for (ConfigFile configFile : metaInfo.getApplication().getConfigFiles()) {
+          retrieveConfigFile(rops, configuration, configFile, name, user,
+              confInstallDir);
+        }
+        for (ConfigFile configFile : clientComponent.getConfigFiles()) {
+          retrieveConfigFile(rops, configuration, configFile, name, user,
+              confInstallDir);
+        }
+      } else {
+        log.info("Installing CLIENT using script {}", client_script);
+        expandAgentTar(agentPkgDir);
+
+        JSONObject commandJson = getCommandJson(defaultConfig, config, metaInfo, clientInstallPath, name);
+        FileWriter file = new FileWriter(new File(cmdDir, "command.json"));
+        try {
+          file.write(commandJson.toString());
+
+        } catch (IOException e) {
+          e.printStackTrace();
+        } finally {
+          file.flush();
+          file.close();
+        }
 
-      runCommand(appPkgDir, agentPkgDir, cmdDir, client_script);
+        runCommand(appPkgDir, agentPkgDir, cmdDir, client_script);
+      }
 
     } catch (IOException ioex) {
       log.warn("Error while executing INSTALL command {}", ioex.getMessage());
       throw new SliderException("INSTALL client failed.");
+    } finally {
+      tmpDir.delete();
     }
   }
 
@@ -481,6 +557,10 @@ public class AgentClientProvider extends AbstractClientProvider
     String libDirProp =
         System.getProperty(SliderKeys.PROPERTY_LIB_DIR);
     File tarFile = new File(libDirProp, SliderKeys.AGENT_TAR);
+    expandTar(tarFile, agentPkgDir);
+  }
+
+  private void expandTar(File tarFile, File destDir) throws IOException {
     TarArchiveInputStream tarIn = new TarArchiveInputStream(
         new GzipCompressorInputStream(
             new BufferedInputStream(
@@ -491,7 +571,7 @@ public class AgentClientProvider extends AbstractClientProvider
     try {
       TarArchiveEntry tarEntry = tarIn.getNextTarEntry();
       while (tarEntry != null) {
-        File destPath = new File(agentPkgDir, tarEntry.getName());
+        File destPath = new File(destDir, tarEntry.getName());
         if (tarEntry.isDirectory()) {
           destPath.mkdirs();
         } else {
@@ -515,6 +595,15 @@ public class AgentClientProvider extends AbstractClientProvider
     }
   }
 
+  private void retrieveConfigFile(RegistryOperations rops,
+      Configuration configuration, ConfigFile configFile, String name,
+      String user, File destDir) throws IOException, SliderException {
+    PublishedConfiguration published = ClientUtils.getConfigFromRegistry(rops,
+        configuration, configFile.getDictionaryName(), name, user, true);
+    ClientUtils.saveOrReturnConfig(published, configFile.getType(),
+        destDir, configFile.getFileName());
+  }
+
   protected JSONObject getCommandJson(JSONObject defaultConfig,
                                       JSONObject inputConfig,
                                       Metainfo metainfo,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/bafc5e72/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
index 24f161f..01a3f1a 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentKeys.java
@@ -48,6 +48,7 @@ public interface AgentKeys {
    */
   String APP_HOME = "app.home";
   String APP_ROOT = "site.global.app_root";
+  String APP_CLIENT_ROOT = "client_root";
   /**
    * Runas user of the application
    */
@@ -77,6 +78,8 @@ public interface AgentKeys {
   String APP_RESOURCES = "application.resources";
   String APP_RESOURCES_DIR = "app/resources";
 
+  String APP_CONF_DIR = "app/conf";
+
   String AGENT_INSTALL_DIR = "infra/agent";
   String APP_DEFINITION_DIR = "app/definition";
   String ADDON_DEFINITION_DIR = "addon/definition";

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/bafc5e72/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
index e368129..ab4da47 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentProviderService.java
@@ -839,7 +839,8 @@ public class AgentProviderService extends AbstractProviderService implements
             SliderKeys.RESOURCE_DIR + "/" + destFile.getName(),
             configResource, fileName);
       } else {
-        launcher.addLocalResource(fileName, configResource);
+        launcher.addLocalResource(AgentKeys.APP_CONF_DIR + "/" + fileName,
+            configResource);
       }
     }
   }
@@ -3127,6 +3128,7 @@ public class AgentProviderService extends AbstractProviderService implements
     config.put("app_log_dir", "${AGENT_LOG_ROOT}");
     config.put("app_pid_dir", "${AGENT_WORK_ROOT}/app/run");
     config.put("app_install_dir", "${AGENT_WORK_ROOT}/app/install");
+    config.put("app_conf_dir", "${AGENT_WORK_ROOT}/" + AgentKeys.APP_CONF_DIR);
     config.put("app_input_conf_dir", "${AGENT_WORK_ROOT}/" + SliderKeys.PROPAGATED_CONF_DIR_NAME);
     config.put("app_container_id", containerId);
     config.put("app_container_tag", tags.getTag(roleName, containerId));

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/bafc5e72/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider2.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider2.java b/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider2.java
index b919fcf..b959e2f 100644
--- a/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider2.java
+++ b/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentClientProvider2.java
@@ -239,7 +239,11 @@ public class TestAgentClientProvider2 extends SliderTestUtils {
   public void testSliderClientForInstallFailures() throws Exception {
     describe(" IGNORE ANY STACK TRACES BELOW ");
 
-    SliderClient client = new SliderClient();
+    SliderClient client = PowerMock.createPartialMock(SliderClient.class,
+        "getRegistryOperations");
+    expect(client.getRegistryOperations()).andReturn(null).anyTimes();
+    PowerMock.replay(SliderClient.class);
+
     client.bindArgs(new Configuration(), "client", "--dest", "a_random_path/none", "--package", "a_random_pkg.zip");
     ActionClientArgs args = new ActionClientArgs();
 
@@ -287,5 +291,7 @@ public class TestAgentClientProvider2 extends SliderTestUtils {
       assertExceptionDetails(e, SliderExitCodes.EXIT_BAD_CONFIGURATION,
           AgentClientProvider.E_COULD_NOT_READ_METAINFO);
     }
+
+    PowerMock.verify(SliderClient.class);
   }
 }