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/04/29 22:09:17 UTC

[1/2] incubator-slider git commit: SLIDER-1107 initial implementation of AM config generation

Repository: incubator-slider
Updated Branches:
  refs/heads/feature/SLIDER-1107_AM_config_generation [created] b1cc78a63


SLIDER-1107 initial implementation of AM config generation


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

Branch: refs/heads/feature/SLIDER-1107_AM_config_generation
Commit: 84fee8a30e4626ef50ba7a44fc4283d12603c389
Parents: 1b833e5
Author: Billie Rinaldi <bi...@gmail.com>
Authored: Fri Apr 29 12:59:25 2016 -0700
Committer: Billie Rinaldi <bi...@gmail.com>
Committed: Fri Apr 29 12:59:25 2016 -0700

----------------------------------------------------------------------
 .../src/main/python/agent/YarnDockerManager.py  |  37 +---
 .../python/resource_management/core/shell.py    |  20 +--
 .../python/scripts/shell_cmd/basic_installer.py |  11 +-
 .../org/apache/slider/client/SliderClient.java  | 110 +++++++++++-
 .../apache/slider/client/SliderClientAPI.java   |  13 +-
 .../org/apache/slider/common/SliderKeys.java    |   3 +-
 .../common/params/ActionResourceArgs.java       |  68 +++++++
 .../apache/slider/common/params/Arguments.java  |   1 +
 .../apache/slider/common/params/ClientArgs.java |  10 ++
 .../slider/common/params/SliderActions.java     |   2 +
 .../slider/common/tools/CoreFileSystem.java     |  60 +++++++
 .../slider/core/launch/AbstractLauncher.java    |  17 ++
 .../core/persist/AppDefinitionPersister.java    |   6 +-
 .../core/registry/docstore/ConfigFormat.java    |   2 +
 .../core/registry/docstore/ConfigUtils.java     |  59 +++++++
 .../docstore/PublishedConfiguration.java        |  23 ++-
 .../PublishedConfigurationOutputter.java        |  75 +++++++-
 .../slider/providers/agent/AgentKeys.java       |   3 +
 .../providers/agent/AgentProviderService.java   | 175 +++++++++++++++++--
 .../application/metadata/AbstractComponent.java |   4 +
 .../agent/application/metadata/Application.java |   4 +
 .../agent/application/metadata/Component.java   |  12 +-
 .../agent/application/metadata/Metainfo.java    |  13 ++
 .../application/metadata/MetainfoParser.java    |  19 +-
 .../agent/TestAgentProviderService.java         |  62 +++++--
 .../metadata/TestMetainfoParser.java            |  13 +-
 .../agent/application/metadata/metainfo.xml     |  21 +++
 27 files changed, 739 insertions(+), 104 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-agent/src/main/python/agent/YarnDockerManager.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/main/python/agent/YarnDockerManager.py b/slider-agent/src/main/python/agent/YarnDockerManager.py
index abdb2fb..4ae19ab 100644
--- a/slider-agent/src/main/python/agent/YarnDockerManager.py
+++ b/slider-agent/src/main/python/agent/YarnDockerManager.py
@@ -55,8 +55,7 @@ class YarnDockerManager(Script):
         self.container_id = command['hostLevelParams']['container_id']
 
     if command['roleCommand'] == 'INSTALL':
-      with Environment(self.workroot) as env:
-        self.install_container(command, env)
+      # do nothing
       returncode = 0; out = ''; err = ''
     if command['roleCommand'] == 'START':
       returncode, out, err = self.start_container(command)    
@@ -130,40 +129,6 @@ class YarnDockerManager(Script):
         return command['configurations']['global'][keyName]
     return None
 
-  def install_container(self, command, env):
-    try:
-      configFiles = self.extract_config_files_from_command(command)
-      for configFile in configFiles:
-        properties = self.extract_config_file_properties_from_command(command, configFile)
-        fileName = configFile['fileName']
-        dir = self.get_config_file_global(command, configFile, 'destDir')
-        if dir is None:
-          dir = self.workroot
-        logger.info("creating config file " + str(configFile) + " in directory "+str(dir))
-        Directory(dir, recursive = True)
-        if configFile['type'] == 'properties':
-          PropertiesFile(fileName,
-                         dir=dir,
-                         properties=properties)
-        elif configFile['type'] == 'env':
-          content = self.get_config_file_global(command, configFile, 'content', useEnv=False)
-          if content is not None:
-            File(os.path.join(dir, fileName),
-                 content=InlineTemplate(content, **properties))
-        elif configFile['type'] == 'template':
-          templateFile = self.get_config_file_global(command, configFile, 'templateFile')
-          if templateFile is not None:
-            with open(templateFile,"r") as fp:
-              fileContent = fp.read()
-            File(os.path.join(dir, fileName),
-                 content=InlineTemplate(fileContent, **properties))
-        elif configFile['type'] == 'xml':
-          XmlConfig(fileName,
-                    conf_dir=dir,
-                    configurations=properties)
-    except:
-      traceback.print_exc()
-
   def start_container(self, command):
     #extracting param needed by docker run from the command passed from AM
     startCommand = self.extract_config_from_command(command, 'docker.startCommand')

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-agent/src/main/python/resource_management/core/shell.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/main/python/resource_management/core/shell.py b/slider-agent/src/main/python/resource_management/core/shell.py
index f21dbbf..e717ae4 100644
--- a/slider-agent/src/main/python/resource_management/core/shell.py
+++ b/slider-agent/src/main/python/resource_management/core/shell.py
@@ -31,10 +31,8 @@ from exceptions import ExecuteTimeoutException
 from resource_management.core.logger import Logger
 import time
 
-APPLICATION_STD_OUTPUT_LOG_FILE_PREFIX = 'application-'
-APPLICATION_STD_OUTPUT_LOG_FILE_FILE_TYPE = '.log'
-APPLICATION_STD_ERROR_LOG_FILE_PREFIX = 'application-'
-APPLICATION_STD_ERROR_LOG_FILE_FILE_TYPE = '.err'
+APPLICATION_STD_OUTPUT_LOG_FILE = 'application.log'
+APPLICATION_STD_ERROR_LOG_FILE = 'application.err'
 
 def checked_call(command, logoutput=False, 
          cwd=None, env=None, preexec_fn=None, user=None, wait_for_finish=True, timeout=None, pid_file=None, poll_after=None):
@@ -69,15 +67,9 @@ def _call(command, logoutput=False, throw_on_failure=True,
   """
   command = ["/bin/bash","--login","-c", command]
   #adding redirecting stdout stderr to file
-  outfilename = APPLICATION_STD_OUTPUT_LOG_FILE_PREFIX + \
-                    str(pid_file_name) + APPLICATION_STD_OUTPUT_LOG_FILE_FILE_TYPE
-          
-  errfilename = APPLICATION_STD_ERROR_LOG_FILE_PREFIX + \
-                    str(pid_file_name) + APPLICATION_STD_ERROR_LOG_FILE_FILE_TYPE
-
-  stdoutFile = open(outfilename, 'w')
-  stderrFile = open(errfilename, 'w')
-  
+  stdoutFile = open(APPLICATION_STD_OUTPUT_LOG_FILE, 'w+')
+  stderrFile = open(APPLICATION_STD_ERROR_LOG_FILE, 'w+')
+
   proc = subprocess.Popen(command, stdout = stdoutFile, stderr = stderrFile, universal_newlines = True,
                           cwd=cwd, env=env, shell=False,
                           preexec_fn=preexec_fn)
@@ -142,4 +134,4 @@ def quote_bash_args(command):
   for char in command:
     if char not in valid:
       return "'" + command.replace("'", "'\"'\"'") + "'"
-  return command
\ No newline at end of file
+  return command

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-agent/src/main/python/scripts/shell_cmd/basic_installer.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/main/python/scripts/shell_cmd/basic_installer.py b/slider-agent/src/main/python/scripts/shell_cmd/basic_installer.py
index 561fd6c..df9b6f0 100644
--- a/slider-agent/src/main/python/scripts/shell_cmd/basic_installer.py
+++ b/slider-agent/src/main/python/scripts/shell_cmd/basic_installer.py
@@ -35,16 +35,15 @@ class BasicInstaller(Script):
     self.configure(env)
     process_cmd = format("{cmd}")
 
-    Execute(process_cmd,
-        logoutput=False,
-        wait_for_finish=False,
-        pid_file=params.pid_file,
-        poll_after = 5
-    )
+    Execute(process_cmd)
 
   def stop(self, env):
     import params
     env.set_params(params)
+    self.configure(env)
+    process_cmd = format("{cmd}")
+
+    Execute(process_cmd)
 
   def status(self, env):
     import params

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/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 dd90e46..f332a66 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
@@ -101,6 +101,7 @@ import org.apache.slider.common.params.ActionNodesArgs;
 import org.apache.slider.common.params.ActionPackageArgs;
 import org.apache.slider.common.params.ActionRegistryArgs;
 import org.apache.slider.common.params.ActionResolveArgs;
+import org.apache.slider.common.params.ActionResourceArgs;
 import org.apache.slider.common.params.ActionStatusArgs;
 import org.apache.slider.common.params.ActionThawArgs;
 import org.apache.slider.common.params.ActionTokensArgs;
@@ -177,7 +178,6 @@ import org.slf4j.LoggerFactory;
 
 import java.io.BufferedReader;
 import java.io.File;
-import java.io.FilenameFilter;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -415,7 +415,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       case ACTION_INSTALL_PACKAGE:
         exitCode = actionInstallPkg(serviceArgs.getActionInstallPackageArgs());
         break;
-      
+
       case ACTION_KEYTAB:
         exitCode = actionKeytab(serviceArgs.getActionKeytabArgs());
         break;
@@ -443,7 +443,11 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       case ACTION_RESOLVE:
         exitCode = actionResolve(serviceArgs.getActionResolveArgs());
         break;
-      
+
+      case ACTION_RESOURCE:
+        exitCode = actionResource(serviceArgs.getActionResourceArgs());
+        break;
+
       case ACTION_STATUS:
         exitCode = actionStatus(clusterName, serviceArgs.getActionStatusArgs());
         break;
@@ -1029,7 +1033,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     Path fileInFs = new Path(pkgPath, keytabInfo.keytab );
     log.info("Deleting keytab {}", fileInFs);
     FileSystem sfs = sliderFileSystem.getFileSystem();
-    require(sfs.exists(fileInFs), "No keytab to delete found at %s", fileInFs.toUri());
+    require(sfs.exists(fileInFs), "No keytab to delete found at %s",
+        fileInFs.toUri());
     sfs.delete(fileInFs, false);
 
     return EXIT_SUCCESS;
@@ -1105,6 +1110,103 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   }
 
   @Override
+  public int actionResource(ActionResourceArgs resourceInfo)
+      throws YarnException, IOException {
+    if (resourceInfo.install) {
+      return actionInstallResource(resourceInfo);
+    } else if (resourceInfo.delete) {
+      return actionDeleteResource(resourceInfo);
+    } else if (resourceInfo.list) {
+      return actionListResource(resourceInfo);
+    } else {
+      throw new BadCommandArgumentsException(
+          "Resource option specified not found.\n"
+              + CommonArgs.usage(serviceArgs, ACTION_RESOURCE));
+    }
+  }
+
+  private int actionListResource(ActionResourceArgs resourceInfo) throws IOException {
+    String folder = resourceInfo.folder != null ? resourceInfo.folder : StringUtils.EMPTY;
+    Path path = sliderFileSystem.buildResourcePath(folder);
+    RemoteIterator<LocatedFileStatus> files =
+        sliderFileSystem.getFileSystem().listFiles(path, true);
+    log.info("Resources:");
+    while (files.hasNext()) {
+      log.info("\t" + files.next().getPath().toString());
+    }
+
+    return EXIT_SUCCESS;
+  }
+
+  private int actionDeleteResource(ActionResourceArgs resourceInfo)
+      throws BadCommandArgumentsException, IOException {
+    if (StringUtils.isEmpty(resourceInfo.resource)) {
+      throw new BadCommandArgumentsException("A file name is required.");
+    }
+
+    Path fileInFs;
+    if (resourceInfo.folder == null) {
+      fileInFs = sliderFileSystem.buildResourcePath(resourceInfo.resource);
+    } else {
+      fileInFs = sliderFileSystem.buildResourcePath(resourceInfo.folder,
+          resourceInfo.resource);
+    }
+
+    log.info("Deleting resource {}", fileInFs);
+    FileSystem sfs = sliderFileSystem.getFileSystem();
+    require(sfs.exists(fileInFs), "No resource to delete found at %s", fileInFs.toUri());
+    sfs.delete(fileInFs, true);
+
+    return EXIT_SUCCESS;
+  }
+
+  private int actionInstallResource(ActionResourceArgs resourceInfo)
+      throws BadCommandArgumentsException, IOException {
+    Path srcFile = null;
+    String folder = resourceInfo.folder != null ? resourceInfo.folder : StringUtils.EMPTY;
+
+    requireArgumentSet(Arguments.ARG_RESOURCE, resourceInfo.resource);
+    File file = new File(resourceInfo.resource);
+    require(file.isFile() || file.isDirectory(),
+        "Unable to access supplied file at %s", file.getAbsolutePath());
+
+    File[] files;
+    if (file.isDirectory()) {
+      files = file.listFiles();
+    } else {
+      files = new File[] { file };
+    }
+
+    Path pkgPath = sliderFileSystem.buildResourcePath(folder);
+    FileSystem sfs = sliderFileSystem.getFileSystem();
+
+    if (!sfs.exists(pkgPath)) {
+      sfs.mkdirs(pkgPath);
+      sfs.setPermission(pkgPath, new FsPermission(
+          FsAction.ALL, FsAction.NONE, FsAction.NONE));
+    } else {
+      require(sfs.isDirectory(pkgPath), "Specified folder %s exists and is " +
+          "not a directory", folder);
+    }
+
+    for (File f : files) {
+      srcFile = new Path(f.toURI());
+
+      Path fileInFs = new Path(pkgPath, srcFile.getName());
+      log.info("Installing file {} at {} and overwrite is {}.",
+          srcFile, fileInFs, resourceInfo.overwrite);
+      require(!(sfs.exists(fileInFs) && !resourceInfo.overwrite),
+          "File exists at %s. Use --overwrite to overwrite.", fileInFs.toUri());
+
+      sfs.copyFromLocalFile(false, resourceInfo.overwrite, srcFile, fileInFs);
+      sfs.setPermission(fileInFs,
+          new FsPermission(FsAction.READ_WRITE, FsAction.NONE, FsAction.NONE));
+    }
+
+    return EXIT_SUCCESS;
+  }
+
+  @Override
   public int actionClient(ActionClientArgs clientInfo) throws
       YarnException,
       IOException {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java b/slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java
index 5c5d96b..30f6ba9 100644
--- a/slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java
+++ b/slider-core/src/main/java/org/apache/slider/client/SliderClientAPI.java
@@ -43,6 +43,7 @@ import org.apache.slider.common.params.ActionKillContainerArgs;
 import org.apache.slider.common.params.ActionListArgs;
 import org.apache.slider.common.params.ActionRegistryArgs;
 import org.apache.slider.common.params.ActionResolveArgs;
+import org.apache.slider.common.params.ActionResourceArgs;
 import org.apache.slider.common.params.ActionStatusArgs;
 import org.apache.slider.common.params.ActionThawArgs;
 import org.apache.slider.common.params.ActionUpgradeArgs;
@@ -117,7 +118,6 @@ public interface SliderClientAPI extends Service {
    * @throws YarnException Yarn problems
    * @throws IOException other problems
    * @throws BadCommandArgumentsException bad arguments.
-   * @deprecated use #actionKeytab
    */
   int actionKeytab(ActionKeytabArgs keytabInfo)
       throws YarnException, IOException;
@@ -134,6 +134,17 @@ public interface SliderClientAPI extends Service {
       throws YarnException, IOException;
 
   /**
+   * Manage file resources leveraged by slider
+   *
+   * @param resourceInfo the arguments needed to manage the resource
+   * @throws YarnException Yarn problems
+   * @throws IOException other problems
+   * @throws BadCommandArgumentsException bad arguments.
+   */
+  int actionResource(ActionResourceArgs resourceInfo)
+      throws YarnException, IOException;
+
+  /**
    * Perform client operations such as install or configure
    *
    * @param clientInfo the arguments needed for client operations

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java b/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
index 1d2d5f8..e06c243 100644
--- a/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/common/SliderKeys.java
@@ -116,7 +116,8 @@ public interface SliderKeys extends SliderXmlConfKeys {
   String HISTORY_FILENAME_SUFFIX = "json";
   String HISTORY_FILENAME_PREFIX = "rolehistory-";
   String KEYTAB_DIR = "keytabs";
-  
+  String RESOURCE_DIR = "resources";
+
   /**
    * Filename pattern is required to save in strict temporal order.
    * Important: older files must sort less-than newer files when using

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/common/params/ActionResourceArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ActionResourceArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ActionResourceArgs.java
new file mode 100644
index 0000000..60fcc87
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ActionResourceArgs.java
@@ -0,0 +1,68 @@
+/*
+ * 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.common.params;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+
+@Parameters(commandNames = {SliderActions.ACTION_RESOURCE},
+    commandDescription = SliderActions.DESCRIBE_ACTION_RESOURCE)
+
+public class ActionResourceArgs  extends AbstractActionArgs {
+
+  @Override
+  public String getActionName() {
+    return SliderActions.ACTION_RESOURCE;
+  }
+
+  @Parameter(names = {ARG_INSTALL},
+      description = "Install the resource(s)")
+  public boolean install;
+
+  @Parameter(names = {ARG_DELETE},
+      description = "Delete the file")
+  public boolean delete;
+
+  @Parameter(names = {ARG_LIST},
+      description = "List of installed files")
+  public boolean list;
+
+  @Parameter(names = {ARG_RESOURCE},
+      description = "Name of the file or directory")
+  public String resource;
+
+  @Parameter(names = {ARG_DESTDIR},
+      description = "The name of the folder in which to store the resources")
+  public String folder;
+
+  @Parameter(names = {ARG_OVERWRITE}, description = "Overwrite existing resource(s)")
+  public boolean overwrite = false;
+
+  /**
+   * Get the min #of params expected
+   * @return the min number of params in the {@link #parameters} field
+   */
+  public int getMinParams() {
+    return 0;
+  }
+
+  @Override
+  public int getMaxParams() {
+    return 3;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java b/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
index bac20d7..63c75ee 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/Arguments.java
@@ -103,6 +103,7 @@ public interface Arguments {
   String ARG_PROVIDER = "--provider";
   String ARG_QUEUE = "--queue";
   String ARG_REPLACE_PKG = "--replacepkg";
+  String ARG_RESOURCE = "--resource";
   String ARG_RESOURCES = "--resources";
   String ARG_RES_COMP_OPT = "--rescompopt";
   String ARG_RES_COMP_OPT_SHORT = "--rco";

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java b/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
index 0a658ea..4016cc9 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/ClientArgs.java
@@ -75,6 +75,7 @@ public class ClientArgs extends CommonArgs {
   private final ActionPackageArgs actionPackageArgs = new ActionPackageArgs();
   private final ActionRegistryArgs actionRegistryArgs = new ActionRegistryArgs();
   private final ActionResolveArgs actionResolveArgs = new ActionResolveArgs();
+  private final ActionResourceArgs actionResourceArgs = new ActionResourceArgs();
   private final ActionStatusArgs actionStatusArgs = new ActionStatusArgs();
   private final ActionThawArgs actionThawArgs = new ActionThawArgs();
   private final ActionTokensArgs actionTokenArgs = new ActionTokensArgs();
@@ -116,6 +117,7 @@ public class ClientArgs extends CommonArgs {
         actionPackageArgs,
         actionRegistryArgs,
         actionResolveArgs,
+        actionResourceArgs,
         actionStatusArgs,
         actionThawArgs,
         actionTokenArgs,
@@ -227,6 +229,10 @@ public class ClientArgs extends CommonArgs {
     return actionResolveArgs;
   }
 
+  public ActionResourceArgs getActionResourceArgs() {
+    return actionResourceArgs;
+  }
+
   public ActionStatusArgs getActionStatusArgs() {
     return actionStatusArgs;
   }
@@ -346,6 +352,10 @@ public class ClientArgs extends CommonArgs {
         bindCoreAction(actionResolveArgs);
         break;
 
+      case ACTION_RESOURCE:
+        bindCoreAction(actionResourceArgs);
+        break;
+
       case ACTION_STATUS:
         bindCoreAction(actionStatusArgs);
         break;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java b/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
index aab7c98..e209cdc 100644
--- a/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
+++ b/slider-core/src/main/java/org/apache/slider/common/params/SliderActions.java
@@ -51,6 +51,7 @@ public interface SliderActions {
   String ACTION_RECONFIGURE = "reconfigure";
   String ACTION_REGISTRY = "registry";
   String ACTION_RESOLVE = "resolve";
+  String ACTION_RESOURCE = "resource";
   String ACTION_STATUS = "status";
   String ACTION_THAW = "start";
   String ACTION_TOKENS = "tokens";
@@ -106,6 +107,7 @@ public interface SliderActions {
   		" Deprecated, use '" + ACTION_KEYTAB + " " + ClientArgs.ARG_INSTALL + "'.";
   String DESCRIBE_ACTION_KEYTAB = "Manage a Kerberos keytab file (install, delete, list) in the sub-folder 'keytabs' of the user's Slider base directory";
   String DESCRIBE_ACTION_DIAGNOSTIC = "Diagnose the configuration of the running slider application and slider client";
+  String DESCRIBE_ACTION_RESOURCE = "Manage a file (install, delete, list) in the sub-folder 'resources' of the user's Slider base directory";
 
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/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 6a02367..0d3436b 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
@@ -23,6 +23,7 @@ import com.google.common.base.Preconditions;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -64,6 +65,8 @@ public class CoreFileSystem {
   private static final Logger
     log = LoggerFactory.getLogger(CoreFileSystem.class);
 
+  private static final String UTF_8 = "UTF-8";
+
   protected final FileSystem fileSystem;
   protected final Configuration configuration;
 
@@ -209,6 +212,55 @@ public class CoreFileSystem {
   }
 
   /**
+   * Build up the path string for resource install location -no attempt to
+   * create the directory is made
+   *
+   * @return the path for resource
+   */
+  public Path buildResourcePath(String resourceFolder) {
+    Preconditions.checkNotNull(resourceFolder);
+    Path path = getBaseApplicationPath();
+    return new Path(path, SliderKeys.RESOURCE_DIR + "/" + resourceFolder);
+  }
+
+  /**
+   * Build up the path string for resource install location -no attempt to
+   * create the directory is made
+   *
+   * @return the path for resource
+   */
+  public Path buildResourcePath(String dirName, String fileName) {
+    Preconditions.checkNotNull(dirName);
+    Preconditions.checkNotNull(fileName);
+    Path path = getBaseApplicationPath();
+    return new Path(path, SliderKeys.RESOURCE_DIR + "/" + dirName + "/" + fileName);
+  }
+
+  /**
+   * Build up the path string for cluster resource install location -no
+   * attempt to create the directory is made
+   *
+   * @return the path for resource
+   */
+  public Path buildClusterResourcePath(String clusterName, String component) {
+    Preconditions.checkNotNull(clusterName);
+    Path path = buildClusterDirPath(clusterName);
+    return new Path(path, SliderKeys.RESOURCE_DIR + "/" + component);
+  }
+
+  /**
+   * Build up the path string for cluster resource install location -no
+   * attempt to create the directory is made
+   *
+   * @return the path for resource
+   */
+  public Path buildClusterResourcePath(String clusterName) {
+    Preconditions.checkNotNull(clusterName);
+    Path path = buildClusterDirPath(clusterName);
+    return new Path(path, SliderKeys.RESOURCE_DIR);
+  }
+
+  /**
    * Create the Slider cluster path for a named cluster and all its subdirs
    * This is a directory; a mkdirs() operation is executed
    * to ensure that it is there.
@@ -778,6 +830,14 @@ public class CoreFileSystem {
     out.close();
   }
 
+  public String cat(Path path) throws IOException {
+    FileStatus status = fileSystem.getFileStatus(path);
+    byte[] b = new byte[(int) status.getLen()];
+    FSDataInputStream in = fileSystem.open(path);
+    int count = in.read(b);
+    return new String(b, 0, count, UTF_8);
+  }
+
   /**
    * Create a path that must exist in the cluster fs
    * @param uri uri to create

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java b/slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
index 0348828..efc58b4 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/AbstractLauncher.java
@@ -50,6 +50,7 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 
 /**
  * Launcher of applications: base class
@@ -71,6 +72,7 @@ public abstract class AbstractLauncher extends Configured {
     Records.newRecord(ContainerLaunchContext.class);
   protected final List<String> commands = new ArrayList<>(20);
   protected final Map<String, LocalResource> localResources = new HashMap<>();
+  protected final Map<String, String> mountPaths = new HashMap<>();
   private final Map<String, ByteBuffer> serviceData = new HashMap<>();
   // security
   protected final Credentials credentials;
@@ -135,6 +137,11 @@ public abstract class AbstractLauncher extends Configured {
     localResources.put(subpath, resource);
   }
 
+  public void addLocalResource(String subpath, LocalResource resource, String mountPath) {
+    localResources.put(subpath, resource);
+    mountPaths.put(subpath, mountPath);
+  }
+
   /**
    * Add a set of local resources
    * @param resourceMap map of name:resource to add
@@ -227,6 +234,16 @@ public abstract class AbstractLauncher extends Configured {
       env.put("YARN_CONTAINER_RUNTIME_TYPE", "docker");
       env.put("YARN_CONTAINER_RUNTIME_DOCKER_IMAGE", dockerImage);//if yarnDockerMode, then dockerImage is set
       env.put("YARN_CONTAINER_RUNTIME_DOCKER_RUN_PRIVILEGED_CONTAINER", runPrivilegedContainer);
+      StringBuilder sb = new StringBuilder();
+      for (Entry<String,String> mount : mountPaths.entrySet()) {
+        if (sb.length() > 0) {
+          sb.append(",");
+        }
+        sb.append(mount.getKey());
+        sb.append(":");
+        sb.append(mount.getValue());
+      }
+      env.put("YARN_CONTAINER_RUNTIME_DOCKER_LOCAL_RESOURCE_MOUNTS", sb.toString());
       log.info("yarn docker env var has been set {}", containerLaunchContext.getEnvironment().toString());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/core/persist/AppDefinitionPersister.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/persist/AppDefinitionPersister.java b/slider-core/src/main/java/org/apache/slider/core/persist/AppDefinitionPersister.java
index 2448c72..e96baef 100644
--- a/slider-core/src/main/java/org/apache/slider/core/persist/AppDefinitionPersister.java
+++ b/slider-core/src/main/java/org/apache/slider/core/persist/AppDefinitionPersister.java
@@ -125,7 +125,11 @@ public class AppDefinitionPersister {
       File tempDir = Files.createTempDir();
       File pkgSrcDir = new File(tempDir, "default");
       pkgSrcDir.mkdirs();
-      Files.copy(buildInfo.appMetaInfo, new File(pkgSrcDir, "metainfo.json"));
+      if (buildInfo.appMetaInfo.getName().endsWith(".xml")) {
+        Files.copy(buildInfo.appMetaInfo, new File(pkgSrcDir, "metainfo.xml"));
+      } else {
+        Files.copy(buildInfo.appMetaInfo, new File(pkgSrcDir, "metainfo.json"));
+      }
 
       Path appDirPath = sliderFileSystem.buildAppDefDirPath(clustername);
       log.info("Using default app def path {}", appDirPath.toString());

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java b/slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java
index 12581d7..9e0c095 100644
--- a/slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java
+++ b/slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigFormat.java
@@ -23,7 +23,9 @@ public enum ConfigFormat {
   JSON("json"),
   PROPERTIES("properties"),
   XML("xml"),
+  HADOOP_XML("hadoop-xml"),
   ENV("env"),
+  TEMPLATE("template"),
 //  YAML("yaml");
   ;
   ConfigFormat(String suffix) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigUtils.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigUtils.java b/slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigUtils.java
new file mode 100644
index 0000000..7e6ee5a
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/core/registry/docstore/ConfigUtils.java
@@ -0,0 +1,59 @@
+/*
+ * 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.core.registry.docstore;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class ConfigUtils {
+  public static String replaceProps(Map<String, String> config, String content) {
+    Map<String, String> tokens = new HashMap<>();
+    for (Entry<String, String> entry : config.entrySet()) {
+      tokens.put("${" + entry.getKey() + "}", entry.getValue());
+      tokens.put("{{" + entry.getKey() + "}}", entry.getValue());
+    }
+    String value = content;
+    for (Map.Entry<String,String> token : tokens.entrySet()) {
+      value = value.replaceAll(Pattern.quote(token.getKey()),
+          Matcher.quoteReplacement(token.getValue()));
+    }
+    return value;
+  }
+
+  public static Map<String, String> replacePropsInConfig(
+      Map<String, String> config, Map<String, String> env) {
+    Map<String, String> tokens = new HashMap<>();
+    for (Entry<String, String> entry : env.entrySet()) {
+      tokens.put("${" + entry.getKey() + "}", entry.getValue());
+    }
+    Map<String, String> newConfig = new HashMap<>();
+    for (Entry<String, String> entry : config.entrySet()) {
+      String value = entry.getValue();
+      for (Map.Entry<String,String> token : tokens.entrySet()) {
+        value = value.replaceAll(Pattern.quote(token.getKey()),
+            Matcher.quoteReplacement(token.getValue()));
+      }
+      newConfig.put(entry.getKey(), entry.getValue());
+    }
+    return newConfig;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfiguration.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfiguration.java b/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfiguration.java
index 50b522f..cabdce6 100644
--- a/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfiguration.java
+++ b/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfiguration.java
@@ -20,6 +20,7 @@ package org.apache.slider.core.registry.docstore;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.slider.common.tools.ConfigHelper;
+import org.apache.slider.common.tools.SliderFileSystem;
 import org.apache.slider.core.exceptions.BadConfigException;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.ObjectMapper;
@@ -49,6 +50,9 @@ public class PublishedConfiguration {
 
   public Map<String, String> entries = new HashMap<>();
 
+  public SliderFileSystem fileSystem;
+  public String clusterName;
+
   public PublishedConfiguration() {
   }
 
@@ -85,7 +89,24 @@ public class PublishedConfiguration {
     putValues(ConfigHelper.resolveConfiguration(keysource, valuesource));
   }
 
-  
+  /**
+   * Build a configuration from the entries
+   * @param description configuration description
+   * @param entries entries to put
+   * @param fileSystem Slider file system (source of configuration templates)
+   * @param clusterName cluster name
+   */
+  public PublishedConfiguration(String description,
+      Iterable<Map.Entry<String, String>> entries,
+      SliderFileSystem fileSystem,
+      String clusterName) {
+    this.description = description;
+    putValues(entries);
+    this.fileSystem = fileSystem;
+    this.clusterName = clusterName;
+  }
+
+
   /**
    * Is the configuration empty. This means either that it has not
    * been given any values, or it is stripped down copy set down over the

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java b/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java
index 15ac207..bae9674 100644
--- a/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java
+++ b/slider-core/src/main/java/org/apache/slider/core/registry/docstore/PublishedConfigurationOutputter.java
@@ -23,13 +23,17 @@ import com.google.common.base.Preconditions;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.slider.common.tools.ConfigHelper;
+import org.apache.slider.common.tools.SliderFileSystem;
 
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.StringWriter;
+import java.util.Collections;
+import java.util.Map;
 import java.util.Properties;
 
 /**
@@ -89,12 +93,13 @@ public abstract class PublishedConfigurationOutputter {
    * @param owner owning config
    * @return the outputter
    */
-  
+
   public static PublishedConfigurationOutputter createOutputter(ConfigFormat format,
       PublishedConfiguration owner) {
     Preconditions.checkNotNull(owner);
     switch (format) {
       case XML:
+      case HADOOP_XML:
         return new XmlOutputter(owner);
       case PROPERTIES:
         return new PropertiesOutputter(owner);
@@ -102,11 +107,13 @@ public abstract class PublishedConfigurationOutputter {
         return new JsonOutputter(owner);
       case ENV:
         return new EnvOutputter(owner);
+      case TEMPLATE:
+        return new TemplateOutputter(owner);
       default:
         throw new RuntimeException("Unsupported format :" + format);
     }
   }
-  
+
   public static class XmlOutputter extends PublishedConfigurationOutputter {
 
 
@@ -131,7 +138,7 @@ public abstract class PublishedConfigurationOutputter {
       return configuration;
     }
   }
-  
+
   public static class PropertiesOutputter extends PublishedConfigurationOutputter {
 
     private final Properties properties;
@@ -146,15 +153,15 @@ public abstract class PublishedConfigurationOutputter {
       properties.store(out, "");
     }
 
-    
+
     public String asString() throws IOException {
       StringWriter sw = new StringWriter();
       properties.store(sw, "");
       return sw.toString();
     }
   }
-    
-    
+
+
   public static class JsonOutputter extends PublishedConfigurationOutputter {
 
     public JsonOutputter(PublishedConfiguration owner) {
@@ -190,9 +197,63 @@ public abstract class PublishedConfigurationOutputter {
         throw new IOException("Configuration has no content field and cannot " +
             "be retrieved as type 'env'");
       }
-      return owner.entries.get("content");
+      String content = owner.entries.get("content");
+      return ConfigUtils.replaceProps(owner.entries, content);
     }
   }
 
+  public static class TemplateOutputter extends PublishedConfigurationOutputter {
+
+    public static final String TEMPLATE_FILE = "template.file";
+
+    public TemplateOutputter(PublishedConfiguration owner) {
+      super(owner);
+    }
+
+    @Override
+    public void save(File dest) throws IOException {
+      FileUtils.writeStringToFile(dest, asString(dest.getName()),
+          Charsets.UTF_8);
+    }
+
+    public String asString(String fileName) throws IOException {
+      if (owner.fileSystem == null) {
+        throw new IOException("File system not specified for template " +
+            "configuration");
+      }
+      Map<String,String> config = owner.entries;
+      SliderFileSystem fileSystem = owner.fileSystem;
+      Path templateFile = null;
+      if (config.containsKey(TEMPLATE_FILE)) {
+        templateFile = fileSystem.buildResourcePath(config.get(TEMPLATE_FILE));
+        if (!fileSystem.isFile(templateFile)) {
+          templateFile = fileSystem.buildResourcePath(owner.clusterName,
+              config.get(TEMPLATE_FILE));
+        }
+        if (!fileSystem.isFile(templateFile)) {
+          throw new IOException("config specified template file " + config
+              .get(TEMPLATE_FILE) + " for config " + owner.description +
+              " but " + templateFile + " doesn't exist");
+        }
+      }
+      if (templateFile == null && fileName != null) {
+        templateFile = fileSystem.buildResourcePath(fileName);
+        if (!fileSystem.isFile(templateFile)) {
+          templateFile = fileSystem.buildResourcePath(owner.clusterName,
+              fileName);
+        }
+      }
+      if (fileSystem.isFile(templateFile)) {
+        return ConfigUtils.replaceProps(config, fileSystem.cat(templateFile));
+      } else {
+        return "";
+      }
+    }
+
+    @Override
+    public String asString() throws IOException {
+      return asString(null);
+    }
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/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 b027939..063ee77 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
@@ -82,6 +82,9 @@ public interface AgentKeys {
   String ADDON_DEFINITION_DIR = "addon/definition";
   String AGENT_CONFIG_FILE = "infra/conf/agent.ini";
   String AGENT_VERSION_FILE = "infra/version";
+  String APP_PACKAGES_DIR = "app/packages";
+  String PER_COMPONENT = "per.component";
+  String PER_GROUP = "per.group";
 
   String JAVA_HOME = "java_home";
   String PACKAGE_LIST = "package_list";

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/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 d5fae19..d2a7e2c 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
@@ -60,8 +60,11 @@ import org.apache.slider.core.exceptions.NoSuchNodeException;
 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.core.registry.docstore.PublishedConfiguration;
+import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
 import org.apache.slider.core.registry.docstore.PublishedExports;
 import org.apache.slider.core.registry.info.CustomRegistryConstants;
 import org.apache.slider.providers.AbstractProviderService;
@@ -124,7 +127,6 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Scanner;
 import java.util.Set;
 import java.util.TreeMap;
@@ -132,7 +134,6 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.regex.Pattern;
 
 import static org.apache.slider.server.appmaster.web.rest.RestPaths.SLIDER_PATH_AGENTS;
 
@@ -435,6 +436,26 @@ public class AgentProviderService extends AbstractProviderService implements
         LocalResourceType.ARCHIVE);
     launcher.addLocalResource(AgentKeys.APP_DEFINITION_DIR, appDefRes);
 
+    for (Package pkg : getMetaInfo().getApplication().getPackages()) {
+      Path pkgPath = fileSystem.buildResourcePath(pkg.getName());
+      if (!fileSystem.isFile(pkgPath)) {
+        pkgPath = fileSystem.buildResourcePath(getClusterName(),
+            pkg.getName());
+      }
+      if (!fileSystem.isFile(pkgPath)) {
+        throw new IOException("Package doesn't exist as a resource: " +
+            pkg.getName());
+      }
+      log.info("Adding resource {}", pkg.getName());
+      LocalResourceType type = LocalResourceType.FILE;
+      if ("archive".equals(pkg.getType())) {
+        type = LocalResourceType.ARCHIVE;
+      }
+      LocalResource packageResource = fileSystem.createAmResource(
+          pkgPath, type);
+      launcher.addLocalResource(AgentKeys.APP_PACKAGES_DIR, packageResource);
+    }
+
     String agentConf = instanceDefinition.getAppConfOperations().
         getGlobalOptions().getOption(AgentKeys.AGENT_CONF, "");
     if (SliderUtils.isSet(agentConf)) {
@@ -476,6 +497,13 @@ public class AgentProviderService extends AbstractProviderService implements
         generatedConfPath,
         SliderKeys.PROPAGATED_CONF_DIR_NAME));
 
+    // build and localize configuration files
+    Map<String, Map<String, String>> configurations =
+        buildCommandConfigurations(instanceDefinition.getAppConfOperations(),
+            container.getId().toString(), roleName, roleGroup);
+    localizeConfigFiles(launcher, roleName, roleGroup, getMetaInfo(),
+        configurations, launcher.getEnv(), fileSystem);
+
     String label = getContainerLabel(container, roleName, roleGroup);
     CommandLineBuilder operation = new CommandLineBuilder();
 
@@ -646,11 +674,42 @@ public class AgentProviderService extends AbstractProviderService implements
   private Path uploadSecurityResource(File resource, SliderFileSystem fileSystem)
       throws IOException {
     Path certsDir = fileSystem.buildClusterSecurityDirPath(getClusterName());
-    if (!fileSystem.getFileSystem().exists(certsDir)) {
-      fileSystem.getFileSystem().mkdirs(certsDir,
+    return uploadResource(resource, fileSystem, certsDir);
+  }
+
+  private Path checkResourceExists(File resource, SliderFileSystem
+      fileSystem, String roleName) throws IOException {
+    Path dir;
+    if (roleName == null) {
+      dir = fileSystem.buildClusterResourcePath(getClusterName());
+    } else {
+      dir = fileSystem.buildClusterResourcePath(getClusterName(), roleName);
+    }
+    Path destPath = new Path(dir, resource.getName());
+    if (fileSystem.getFileSystem().exists(destPath)) {
+      return destPath;
+    }
+    return null;
+  }
+
+  private Path uploadResource(File resource, SliderFileSystem fileSystem,
+      String roleName) throws IOException {
+    Path dir;
+    if (roleName == null) {
+      dir = fileSystem.buildClusterResourcePath(getClusterName());
+    } else {
+      dir = fileSystem.buildClusterResourcePath(getClusterName(), roleName);
+    }
+    return uploadResource(resource, fileSystem, dir);
+  }
+
+  private static Path uploadResource(File resource, SliderFileSystem fileSystem,
+                              Path parentDir) throws IOException {
+    if (!fileSystem.getFileSystem().exists(parentDir)) {
+      fileSystem.getFileSystem().mkdirs(parentDir,
         new FsPermission(FsAction.ALL, FsAction.NONE, FsAction.NONE));
     }
-    Path destPath = new Path(certsDir, resource.getName());
+    Path destPath = new Path(parentDir, resource.getName());
     if (!fileSystem.getFileSystem().exists(destPath)) {
       FSDataOutputStream os = fileSystem.getFileSystem().create(destPath);
       byte[] contents = FileUtils.readFileToByteArray(resource);
@@ -659,6 +718,9 @@ public class AgentProviderService extends AbstractProviderService implements
       os.flush();
       os.close();
       log.info("Uploaded {} to localization path {}", resource, destPath);
+    } else {
+      log.info("Resource {} already existed at localization path {}", resource,
+          destPath);
     }
 
     while (!fileSystem.getFileSystem().exists(destPath)) {
@@ -718,6 +780,68 @@ public class AgentProviderService extends AbstractProviderService implements
     }
   }
 
+  private void createConfigFile(SliderFileSystem fileSystem, File file,
+      ConfigFile configFile, Map<String, String> config) throws IOException {
+    ConfigFormat configFormat = ConfigFormat.resolve(configFile.getType());
+    log.info("Writing {} file {}", configFormat, file);
+
+    PublishedConfigurationOutputter configurationOutputter =
+      PublishedConfigurationOutputter.createOutputter(configFormat,
+          new PublishedConfiguration(configFile.getDictionaryName(),
+              config.entrySet(), fileSystem, getClusterName()));
+    configurationOutputter.save(file);
+  }
+
+  @VisibleForTesting
+  protected void localizeConfigFiles(ContainerLauncher launcher,
+                                     String roleName, String roleGroup,
+                                     Metainfo metainfo,
+                                     Map<String, Map<String, String>> configs,
+                                     MapOperations env,
+                                     SliderFileSystem fileSystem)
+      throws IOException {
+    for (ConfigFile configFile : metainfo.getComponentConfigFiles(roleGroup)) {
+      Map<String, String> config = ConfigUtils.replacePropsInConfig(
+          configs.get(configFile.getDictionaryName()), env.options);
+      String fileName = ConfigUtils.replaceProps(config,
+          configFile.getFileName());
+      File localFile = new File(SliderKeys.RESOURCE_DIR);
+      if (!localFile.exists()) {
+        localFile.mkdir();
+      }
+      localFile = new File(localFile, new File(fileName).getName());
+
+      String folder = null;
+      if ("true".equals(config.get(PER_COMPONENT))) {
+        folder = roleName;
+      } else if ("true".equals(config.get(PER_GROUP))) {
+        folder = roleGroup;
+      }
+
+      Path destPath = checkResourceExists(localFile, fileSystem, folder);
+      if (destPath == null) {
+        log.info("Localizing {} configs to config file {} (destination {}) " +
+                "based on {} configs",
+            config.size(), localFile, fileName, configFile.getDictionaryName());
+        createConfigFile(fileSystem, localFile, configFile, config);
+        destPath = uploadResource(localFile, fileSystem, folder);
+      } else {
+        log.info("Config already exists at {}, not recreating it", destPath);
+      }
+      LocalResource configResource = fileSystem.createAmResource(destPath,
+          LocalResourceType.FILE);
+
+      File destFile = new File(fileName);
+      if (destFile.isAbsolute()) {
+        launcher.addLocalResource(
+            SliderKeys.RESOURCE_DIR + "/" + destFile.getName(),
+            configResource, fileName);
+      } else {
+        launcher.addLocalResource(fileName, configResource);
+      }
+    }
+  }
+
   /**
    * build the zookeeper registry path.
    * 
@@ -2036,7 +2160,7 @@ public class AgentProviderService extends AbstractProviderService implements
     cmd.setConfigurations(configurations);
     Map<String, Map<String, String>> componentConfigurations = buildComponentConfigurations(appConf);
     cmd.setComponentConfigurations(componentConfigurations);
-    
+
     if (SliderUtils.isSet(scriptPath)) {
       cmd.setCommandParams(commandParametersSet(scriptPath, timeout, false));
     } else {
@@ -2154,10 +2278,10 @@ public class AgentProviderService extends AbstractProviderService implements
     List<String> packages = new ArrayList<>();
     if (application != null) {
       if (application.getPackages().size() > 0) {
-        List<Package> appPackages = application.getPackages();
-        for (Package appPackage : appPackages) {
-          packages.add(String.format(pkgFormatString, appPackage.getType(), appPackage.getName()));
-        }
+        // no-op if there are packages that are not OS-specific, as these
+        // will be localized by AM rather than the Agent
+        // this should be backwards compatible, as there was previously an
+        // XML parsing bug that ensured non-OS-specific packages did not exist
       } else {
         List<OSSpecific> osSpecifics = application.getOSSpecifics();
         if (osSpecifics != null && osSpecifics.size() > 0) {
@@ -2821,14 +2945,41 @@ public class AgentProviderService extends AbstractProviderService implements
       }
     }
 
+    boolean finished = false;
+    while (!finished) {
+      finished = true;
+      for (Map.Entry<String, String> entry : allConfigs.entrySet()) {
+        String configValue = entry.getValue();
+        for (Map.Entry<String, String> lookUpEntry : allConfigs.entrySet()) {
+          String lookUpValue = lookUpEntry.getValue();
+          if (lookUpValue.contains("${@//site/")) {
+            continue;
+          }
+          String lookUpKey = lookUpEntry.getKey();
+          if (configValue != null && configValue.contains(lookUpKey)) {
+            configValue = configValue.replace(lookUpKey, lookUpValue);
+          }
+        }
+        if (!configValue.equals(entry.getValue())) {
+          finished = false;
+          allConfigs.put(entry.getKey(), configValue);
+        }
+      }
+    }
+
     for (String configType : configurations.keySet()) {
       Map<String, String> configBucket = configurations.get(configType);
       for (Map.Entry<String, String> entry: configBucket.entrySet()) {
         String configName = entry.getKey();
         String configValue = entry.getValue();
-        for (String lookUpKey : allConfigs.keySet()) {
+        for (Map.Entry<String, String> lookUpEntry : allConfigs.entrySet()) {
+          String lookUpValue = lookUpEntry.getValue();
+          if (lookUpValue.contains("${@//site/")) {
+            continue;
+          }
+          String lookUpKey = lookUpEntry.getKey();
           if (configValue != null && configValue.contains(lookUpKey)) {
-            configValue = configValue.replace(lookUpKey, allConfigs.get(lookUpKey));
+            configValue = configValue.replace(lookUpKey, lookUpValue);
           }
         }
         configBucket.put(configName, configValue);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractComponent.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractComponent.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractComponent.java
index 1b63b58..b6ae4de 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractComponent.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/AbstractComponent.java
@@ -65,6 +65,10 @@ public abstract class AbstractComponent implements Validate {
     this.commands = commands;
   }
 
+  public void addCommand(ComponentCommand command) {
+    commands.add(command);
+  }
+
   @Override
   public String toString() {
     final StringBuilder sb = new StringBuilder("{");

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java
index 63546a4..5556c7f 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Application.java
@@ -116,6 +116,10 @@ public class Application extends AbstractMetainfoSchema {
     return commandOrders;
   }
 
+  public void addPackage(Package pkg) {
+    packages.add(pkg);
+  }
+
   @JsonProperty("packages")
   public List<Package> getPackages() {
     return packages;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java
index 3f23455..78bb8c1 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Component.java
@@ -38,7 +38,8 @@ public class Component extends AbstractComponent {
   String type = TYPE_STANDARD;
   List<ComponentExport> componentExports = new ArrayList<>();
   List<DockerContainer> dockerContainers = new ArrayList<>();
-  
+  List<ConfigFile> configFiles = new ArrayList<>();
+
   public Component() {
   }
 
@@ -155,6 +156,15 @@ public class Component extends AbstractComponent {
     return Boolean.parseBoolean(this.autoStartOnFailure);
   }
 
+  public void addConfigFile(ConfigFile configFile) {
+    this.configFiles.add(configFile);
+  }
+
+  @JsonProperty("configFiles")
+  public List<ConfigFile> getConfigFiles() {
+    return configFiles;
+  }
+
   @Override
   public String toString() {
     final StringBuilder sb =

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
index 036d98e..10c497f 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/Metainfo.java
@@ -21,6 +21,9 @@ import org.apache.slider.core.exceptions.SliderException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * Application metainfo uber class
  */
@@ -71,6 +74,16 @@ public class Metainfo {
     return null;
   }
 
+  public List<ConfigFile> getComponentConfigFiles(String roleGroup) {
+    List<ConfigFile> componentConfigFiles = new ArrayList<>();
+    componentConfigFiles.addAll(application.getConfigFiles());
+    Component component = getApplicationComponent(roleGroup);
+    if (component != null) {
+      componentConfigFiles.addAll(component.getConfigFiles());
+    }
+    return componentConfigFiles;
+  }
+
   public void validate() throws SliderException {
     if (!VERSION_TWO_ONE.equals(schemaVersion) &&
         !VERSION_TWO_ZERO.equals(schemaVersion)) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
index a6f0e9d..8b520eb 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/application/metadata/MetainfoParser.java
@@ -67,14 +67,25 @@ public class MetainfoParser extends AbstractMetainfoParser{
     digester.addBeanPropertySetter("*/commandScript/timeout");
     digester.addSetNext("*/commandScript", "addCommandScript");
 
+    digester.addObjectCreate("*/command", ComponentCommand.class);
+    digester.addBeanPropertySetter("*/command/exec");
+    digester.addBeanPropertySetter("*/command/name");
+    digester.addBeanPropertySetter("*/command/type");
+    digester.addSetNext("*/command", "addCommand");
+
     digester.addObjectCreate("*/osSpecific", OSSpecific.class);
     digester.addBeanPropertySetter("*/osSpecific/osType");
-    digester.addObjectCreate("*/package", OSPackage.class);
-    digester.addBeanPropertySetter("*/package/type");
-    digester.addBeanPropertySetter("*/package/name");
-    digester.addSetNext("*/package", "addOSPackage");
+    digester.addObjectCreate("*/osSpecific/packages/package", OSPackage.class);
+    digester.addBeanPropertySetter("*/osSpecific/packages/package/type");
+    digester.addBeanPropertySetter("*/osSpecific/packages/package/name");
+    digester.addSetNext("*/osSpecific/packages/package", "addOSPackage");
     digester.addSetNext("*/osSpecific", "addOSSpecific");
 
+    digester.addObjectCreate("*/application/packages/package", Package.class);
+    digester.addBeanPropertySetter("*/application/packages/package/type");
+    digester.addBeanPropertySetter("*/application/packages/package/name");
+    digester.addSetNext("*/application/packages/package", "addPackage");
+
     digester.addObjectCreate("*/configFile", ConfigFile.class);
     digester.addBeanPropertySetter("*/configFile/type");
     digester.addBeanPropertySetter("*/configFile/fileName");

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java b/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java
index 9fbb3d0..af1c61f 100644
--- a/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java
+++ b/slider-core/src/test/java/org/apache/slider/providers/agent/TestAgentProviderService.java
@@ -88,11 +88,13 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.ByteArrayInputStream;
+import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -285,6 +287,7 @@ public class TestAgentProviderService {
     ProviderRole role = new ProviderRole("HBASE_MASTER", 1);
     SliderFileSystem sliderFileSystem = createNiceMock(SliderFileSystem.class);
     ContainerLauncher launcher = createNiceMock(ContainerLauncher.class);
+    expect(launcher.getEnv()).andReturn(new MapOperations()).anyTimes();
     Path generatedConfPath = new Path(".", "test");
     MapOperations resourceComponent = new MapOperations();
     MapOperations appComponent = new MapOperations();
@@ -347,6 +350,9 @@ public class TestAgentProviderService {
                                                  anyString(),
                                                  anyString()
     );
+
+    doReturn(Collections.emptyMap()).when(mockAps).getRoleClusterNodeMapping();
+
     expect(access.isApplicationLive()).andReturn(true).anyTimes();
     ClusterDescription desc = new ClusterDescription();
     desc.setOption(OptionKeys.ZOOKEEPER_QUORUM, "host1:2181");
@@ -357,16 +363,15 @@ public class TestAgentProviderService {
     expect(access.getRoleClusterNodeMapping()).andReturn(cnMap).anyTimes();
 
 
-    AggregateConf aggConf = new AggregateConf();
-    ConfTreeOperations treeOps = aggConf.getAppConfOperations();
+    ConfTreeOperations treeOps = instanceDefinition.getAppConfOperations();
     treeOps.getOrAddComponent("HBASE_MASTER").put(AgentKeys.WAIT_HEARTBEAT, "0");
     treeOps.set(OptionKeys.APPLICATION_NAME, "HBASE");
     treeOps.set("site.fs.defaultFS", "hdfs://HOST1:8020/");
     treeOps.set("internal.data.dir.path", "hdfs://HOST1:8020/database");
-    expect(access.getInstanceDefinitionSnapshot()).andReturn(aggConf);
+    expect(access.getInstanceDefinitionSnapshot()).andReturn(instanceDefinition);
     expect(access.getInternalsSnapshot()).andReturn(treeOps).anyTimes();
     expect(access.getAppConfSnapshot()).andReturn(treeOps).anyTimes();
-    replay(access, ctx, container, sliderFileSystem, mockFs);
+    replay(access, ctx, container, sliderFileSystem, mockFs, launcher);
 
     try {
       mockAps.buildContainerLaunchContext(launcher,
@@ -1225,6 +1230,8 @@ public class TestAgentProviderService {
     SliderFileSystem sliderFileSystem = createNiceMock(SliderFileSystem.class);
     ContainerLauncher launcher = createNiceMock(ContainerLauncher.class);
     ContainerLauncher launcher2 = createNiceMock(ContainerLauncher.class);
+    expect(launcher.getEnv()).andReturn(new MapOperations()).anyTimes();
+    expect(launcher2.getEnv()).andReturn(new MapOperations()).anyTimes();
     Path generatedConfPath = new Path(".", "test");
     MapOperations resourceComponent = new MapOperations();
     MapOperations appComponent = new MapOperations();
@@ -1291,18 +1298,28 @@ public class TestAgentProviderService {
     desc.setInfo(OptionKeys.APPLICATION_NAME, "HBASE");
     expect(access.getClusterStatus()).andReturn(desc).anyTimes();
 
-    AggregateConf aggConf = new AggregateConf();
-    ConfTreeOperations treeOps = aggConf.getAppConfOperations();
+    ConfTreeOperations treeOps = instanceDefinition.getAppConfOperations();
     treeOps.getOrAddComponent("HBASE_MASTER").put(AgentKeys.WAIT_HEARTBEAT, "0");
-    treeOps.getOrAddComponent("HBASE_REGIONSERVER").put(AgentKeys.WAIT_HEARTBEAT, "0");
+    treeOps.getOrAddComponent("HBASE_REGIONSERVER").put(
+        AgentKeys.WAIT_HEARTBEAT, "0");
     treeOps.set(OptionKeys.APPLICATION_NAME, "HBASE");
     treeOps.set("site.fs.defaultFS", "hdfs://HOST1:8020/");
     treeOps.set("internal.data.dir.path", "hdfs://HOST1:8020/database");
-    expect(access.getInstanceDefinitionSnapshot()).andReturn(aggConf).anyTimes();
+    expect(access.getInstanceDefinitionSnapshot()).andReturn(instanceDefinition).anyTimes();
     expect(access.getInternalsSnapshot()).andReturn(treeOps).anyTimes();
     expect(access.getAppConfSnapshot()).andReturn(treeOps).anyTimes();
-    doNothing().when(mockAps).publishApplicationInstanceData(anyString(), anyString(), anyCollection());
-    replay(access, ctx, container, sliderFileSystem, mockFs);
+    doNothing().when(mockAps).publishApplicationInstanceData(anyString(),
+        anyString(), anyCollection());
+    doNothing().when(mockAps).localizeConfigFiles(
+        (ContainerLauncher)Matchers.anyObject(),
+        anyString(),
+        anyString(),
+        (Metainfo)Matchers.anyObject(),
+        anyMap(),
+        (MapOperations)Matchers.anyObject(),
+        (SliderFileSystem)Matchers.anyObject());
+    doReturn(Collections.emptyMap()).when(mockAps).getRoleClusterNodeMapping();
+    replay(access, ctx, container, sliderFileSystem, mockFs, launcher, launcher2);
 
     // build two containers
     try {
@@ -1849,12 +1866,12 @@ public class TestAgentProviderService {
     AgentProviderService aps = createAgentProviderService(new Configuration());
     Map<String, Map<String, String>> allConfigs = new HashMap<String, Map<String, String>>();
     Map<String, String> cfg1 = new HashMap<String, String>();
-    cfg1.put("a1", "${@//site/cfg-2/A1}");
+    cfg1.put("a1", "0${@//site/cfg-2/A1}");
     cfg1.put("b1", "22");
     cfg1.put("c1", "33");
     cfg1.put("d1", "${@//site/cfg1/c1}AA");
     Map<String, String> cfg2 = new HashMap<String, String>();
-    cfg2.put("A1", "11");
+    cfg2.put("A1", "11${@//site/cfg1/b1}");
     cfg2.put("B1", "${@//site/cfg-2/A1},${@//site/cfg-2/A1},AA,${@//site/cfg1/c1}");
     cfg2.put("C1", "DD${@//site/cfg1/c1}");
     cfg2.put("D1", "${14}");
@@ -1862,15 +1879,30 @@ public class TestAgentProviderService {
     allConfigs.put("cfg1", cfg1);
     allConfigs.put("cfg-2", cfg2);
     aps.dereferenceAllConfigs(allConfigs);
-    Assert.assertEquals("11", cfg1.get("a1"));
+    Assert.assertEquals("01122", cfg1.get("a1"));
     Assert.assertEquals("22", cfg1.get("b1"));
     Assert.assertEquals("33", cfg1.get("c1"));
     Assert.assertEquals("33AA", cfg1.get("d1"));
 
-    Assert.assertEquals("11", cfg2.get("A1"));
-    Assert.assertEquals("11,11,AA,33", cfg2.get("B1"));
+    Assert.assertEquals("1122", cfg2.get("A1"));
+    Assert.assertEquals("1122,1122,AA,33", cfg2.get("B1"));
     Assert.assertEquals("DD33", cfg2.get("C1"));
     Assert.assertEquals("${14}", cfg2.get("D1"));
   }
 
+  @Test
+  public void testDereferenceAllConfigLoop() throws IOException {
+    AgentProviderService aps = createAgentProviderService(new Configuration());
+    Map<String, Map<String, String>> allConfigs = new HashMap<String, Map<String, String>>();
+    Map<String, String> cfg1 = new HashMap<String, String>();
+    cfg1.put("a1", "0${@//site/cfg-2/A1}");
+    Map<String, String> cfg2 = new HashMap<String, String>();
+    cfg2.put("A1", "11${@//site/cfg1/a1}");
+
+    allConfigs.put("cfg1", cfg1);
+    allConfigs.put("cfg-2", cfg2);
+    aps.dereferenceAllConfigs(allConfigs);
+    Assert.assertEquals("0${@//site/cfg-2/A1}", cfg1.get("a1"));
+    Assert.assertEquals("11${@//site/cfg1/a1}", cfg2.get("A1"));
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java b/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
index 30283d1..ba1912a 100644
--- a/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
+++ b/slider-core/src/test/java/org/apache/slider/providers/agent/application/metadata/TestMetainfoParser.java
@@ -50,7 +50,7 @@ public class TestMetainfoParser {
     Assert.assertNotNull(metainfo.getApplication());
     Application application = metainfo.getApplication();
     assert "STORM".equals(application.getName());
-    assert 5 == application.getComponents().size();
+    assert 6 == application.getComponents().size();
     OSPackage pkg = application.getOSSpecifics().get(0).getPackages().get(0);
     assert "tarball".equals(pkg.getType());
     assert "files/apache-storm-0.9.1.2.1.1.0-237.tar.gz".equals(pkg.getName());
@@ -63,9 +63,20 @@ public class TestMetainfoParser {
       if (comp != null && comp.getName().equals("SUPERVISOR")) {
         Assert.assertEquals(1, comp.getComponentExports().size());
       }
+      if (comp != null && comp.getName().equals("ANOTHER_COMPONENT")) {
+        assert 2 == comp.getCommands().size();
+        assert "start command".equals(comp.getCommands().get(0).getExec());
+        assert "START".equals(comp.getCommands().get(0).getName());
+        assert "stop command".equals(comp.getCommands().get(1).getExec());
+        assert "STOP".equals(comp.getCommands().get(1).getName());
+      }
     }
     assert found;
     Assert.assertEquals(0, application.getConfigFiles().size());
+    assert 1 == application.getPackages().size();
+    Package p = application.getPackages().get(0);
+    assert "tarball".equals(p.getType());
+    assert "test-tarball-name.tgz".equals(p.getName());
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/84fee8a3/slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml
----------------------------------------------------------------------
diff --git a/slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml b/slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml
index d9004ad..fbe9299 100644
--- a/slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml
+++ b/slider-core/src/test/resources/org/apache/slider/providers/agent/application/metadata/metainfo.xml
@@ -142,6 +142,20 @@
           <timeout>600</timeout>
         </commandScript>
       </component>
+
+      <component>
+        <name>ANOTHER_COMPONENT</name>
+        <category>MASTER</category>
+        <commands>
+          <command>
+            <exec>start command</exec>
+          </command>
+          <command>
+            <exec>stop command</exec>
+            <name>STOP</name>
+          </command>
+        </commands>
+      </component>
     </components>
 
     <osSpecifics>
@@ -155,5 +169,12 @@
         </packages>
       </osSpecific>
     </osSpecifics>
+
+    <packages>
+      <package>
+        <type>tarball</type>
+        <name>test-tarball-name.tgz</name>
+      </package>
+    </packages>
   </application>
 </metainfo>


[2/2] incubator-slider git commit: SLIDER-1107 initial pkgless hbase implementation

Posted by bi...@apache.org.
SLIDER-1107 initial pkgless hbase implementation


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

Branch: refs/heads/feature/SLIDER-1107_AM_config_generation
Commit: b1cc78a639b9eab2e00584a5f3a90d6536317171
Parents: 84fee8a
Author: Billie Rinaldi <bi...@gmail.com>
Authored: Fri Apr 29 13:01:51 2016 -0700
Committer: Billie Rinaldi <bi...@gmail.com>
Committed: Fri Apr 29 13:01:51 2016 -0700

----------------------------------------------------------------------
 app-packages/hbase-nopkg/README.md              |  55 +++++
 app-packages/hbase-nopkg/appConfig-default.json | 112 +++++++++
 .../hbase-nopkg/appConfig-secured-default.json  | 120 ++++++++++
 app-packages/hbase-nopkg/metainfo-secured.xml   | 235 +++++++++++++++++++
 app-packages/hbase-nopkg/metainfo.xml           | 229 ++++++++++++++++++
 app-packages/hbase-nopkg/pom.xml                |  92 ++++++++
 app-packages/hbase-nopkg/resources-default.json |  47 ++++
 ...-metrics2-hbase.properties-GANGLIA-MASTER.j2 |  62 +++++
 ...doop-metrics2-hbase.properties-GANGLIA-RS.j2 |  62 +++++
 .../hadoop-metrics2-hbase.properties-MASTER.j2  |  47 ++++
 .../hadoop-metrics2-hbase.properties-RS.j2      |  47 ++++
 .../resources/hbase-env-secured.sh.j2           |  63 +++++
 .../hbase-nopkg/resources/hbase-env.sh.j2       |  59 +++++
 .../resources/hbase-log4j.properties.j2         | 110 +++++++++
 .../resources/hbase_client_jaas.conf.j2         |  22 ++
 .../resources/hbase_master_jaas.conf.j2         |  25 ++
 .../resources/hbase_regionserver_jaas.conf.j2   |  25 ++
 app-packages/hbase-nopkg/src/assembly/hbase.xml |  82 +++++++
 18 files changed, 1494 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/README.md
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/README.md b/app-packages/hbase-nopkg/README.md
new file mode 100644
index 0000000..70efa46
--- /dev/null
+++ b/app-packages/hbase-nopkg/README.md
@@ -0,0 +1,55 @@
+<!---
+   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.
+-->
+
+# Create Slider App Package for HBase
+
+appConfig-default.json and resources-default.json are not required to be packaged.
+These files are included as reference configuration for Slider apps and are suitable
+for a one-node cluster.
+
+To create the app package you will need the HBase tarball and invoke mvn command
+with appropriate parameters. It is recommended that pkg.version be set to the
+ same value as hbase.version.
+
+Command:
+
+    mvn clean package -Phbase-resources -Dhbase.version=<hbase version> -Dpkg.version=<app package version>
+       -Dpkg.name=<file name of app tarball> -Dpkg.src=<folder location where the pkg is available>
+
+Example:
+
+    mvn clean package -Phbase-resources -Dhbase.version=1.1.4
+      -Dpkg.version=1.1.4 -Dpkg.name=hbase-1.1.4-bin.tar.gz
+      -Dpkg.src=/Users/user1/Downloads
+
+App package can be found in
+
+    app-packages/hbase-nopkg/target/slider-hbase-resources-1.1.4.zip
+
+## Verifying the content
+
+Verify the content using
+
+    zip -Tv slider-hbase-*.zip
+
+## Sample commands
+
+    unzip slider-hbase-resources-1.1.4.zip
+    slider resource --install --resource resources --destdir hbase
+    slider create hbase --template appConfig-default.json --resources resources-default.json --metainfo metainfo.xml
+
+

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/appConfig-default.json
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/appConfig-default.json b/app-packages/hbase-nopkg/appConfig-default.json
new file mode 100644
index 0000000..b482db9
--- /dev/null
+++ b/app-packages/hbase-nopkg/appConfig-default.json
@@ -0,0 +1,112 @@
+{
+    "schema": "http://example.org/specification/v2.0.0",
+    "metadata": {
+    },
+    "global": {
+        "create.default.zookeeper.node": "true",
+        "java_home": "/usr/jdk64/jdk1.8.0_60",
+        "system_configs": "core-site",
+
+        "site.global.app_user": "yarn",
+        "site.global.app_version": "${hbase.version}",
+        "site.global.app_root": "${AGENT_WORK_ROOT}/app/packages/hbase-${hbase.version}",
+        "site.global.app_conf_dir": "${AGENT_WORK_ROOT}/app/conf",
+
+        "site.global.pid_file": "${@//site/global/app_pid_dir}/hbase-${@//site/global/app_user}-${@//site/global/hbase_component}.pid",
+        "site.global.daemon_cmd": "env HBASE_IDENT_STRING=${@//site/global/app_user} ${@//site/global/app_root}/bin/hbase-daemon.sh --config ${@//site/global/app_conf_dir}",
+
+        "site.global.hbase_instance_name": "instancename",
+        "site.global.hbase_root_password": "secret",
+        "site.global.user_group": "hadoop",
+
+        "site.global.thrift_port": "0",
+        "site.global.thrift2_port": "0",
+        "site.global.rest_port": "0",
+
+        "site.hbase-env.template.file": "hbase-env.sh.j2",
+        "site.hbase-env.java64_home": "${JAVA_HOME}",
+        "site.hbase-env.conf_dir": "${@//site/global/app_conf_dir}",
+        "site.hbase-env.log_dir": "${@//site/global/app_log_dir}",
+        "site.hbase-env.pid_dir": "${@//site/global/app_pid_dir}",
+        "site.hbase-env.master_heapsize": "1024m",
+        "site.hbase-env.regionserver_heapsize": "1024m",
+        "site.hbase-env.regionserver_xmn_size": "512m",
+        "site.hbase-env.hbase_opts": "",
+
+        "site.hbase-site.hbase.rootdir": "${DEFAULT_DATA_DIR}/data",
+        "site.hbase-site.hbase.cluster.distributed": "true",
+        "site.hbase-site.hbase.superuser": "${USER_NAME}",
+        "site.hbase-site.hbase.tmp.dir": "work/app/tmp",
+        "site.hbase-site.hbase.local.dir": "${hbase.tmp.dir}/local",
+        "site.hbase-site.hbase.zookeeper.quorum": "${ZK_HOST}",
+        "site.hbase-site.zookeeper.znode.parent": "${DEFAULT_ZK_PATH}",
+        "site.hbase-site.hbase.regionserver.info.port": "0",
+        "site.hbase-site.hbase.bulkload.staging.dir": "/user/${USER_NAME}/hbase-staging",
+        "site.hbase-site.hbase.coprocessor.region.classes": "org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint",
+        "site.hbase-site.hbase.master.info.bindAddress": "0.0.0.0",
+        "site.hbase-site.hbase.master.info.port": "16010",
+        "site.hbase-site.hbase.regionserver.port": "0",
+        "site.hbase-site.hbase.master.port": "0",
+        "site.hbase-site.hbase.regionserver.global.memstore.upperLimit": "0.4",
+        "site.hbase-site.hbase.regionserver.handler.count": "60",
+        "site.hbase-site.hbase.hregion.majorcompaction": "86400000",
+        "site.hbase-site.hbase.regionserver.global.memstore.lowerLimit": "0.38",
+        "site.hbase-site.hbase.hregion.memstore.block.multiplier": "2",
+        "site.hbase-site.hbase.hregion.memstore.flush.size": "134217728",
+        "site.hbase-site.hbase.hregion.memstore.mslab.enabled": "true",
+        "site.hbase-site.hbase.hregion.max.filesize": "10737418240",
+        "site.hbase-site.hbase.client.scanner.caching": "100",
+        "site.hbase-site.zookeeper.session.timeout": "30000",
+        "site.hbase-site.hbase.client.keyvalue.maxsize": "10485760",
+        "site.hbase-site.hbase.hstore.compactionThreshold": "3",
+        "site.hbase-site.hbase.hstore.flush.retries.number": "120",
+        "site.hbase-site.hbase.hstore.blockingStoreFiles": "10",
+        "site.hbase-site.hfile.block.cache.size": "0.40",
+        "site.hbase-site.hbase.zookeeper.property.clientPort": "2181",
+        "site.hbase-site.hbase.zookeeper.useMulti": "true",
+        "site.hbase-site.hbase.defaults.for.version.skip": "true",
+        "site.hbase-site.dfs.domain.socket.path": "/var/lib/hadoop-hdfs/dn_socket",
+
+        "site.hbase-site.hbase.security.authentication": "simple",
+        "site.hbase-site.hbase.security.authorization": "false",
+        "site.hbase-site.hbase.coprocessor.region.classes": "",
+        "site.hbase-site.hbase.coprocessor.master.classes": "",
+
+        "site.hbase-policy.security.client.protocol.acl": "*",
+        "site.hbase-policy.security.admin.protocol.acl": "*",
+        "site.hbase-policy.security.masterregion.protocol.acl": "*",
+
+        "site.hbase-log4j.template.file": "hbase-log4j.properties.j2",
+
+        "site.hbase-metrics2.template.file": "hadoop-metrics2-hbase.properties-RS.j2",
+        "site.hbase-metrics2.app_name": "${CLUSTER_NAME}",
+        "site.hbase-metrics2.metric_collector_host": "${NN_HOST}",
+        "site.hbase-metrics2.metric_collector_port": "6188",
+        "site.hbase-metrics2.metric_collector_lib": ""
+    },
+    "components": {
+        "slider-appmaster": {
+            "jvm.heapsize": "1024M"
+        },
+        "HBASE_MASTER": {
+            "site.global.hbase_component": "master",
+            "site.hbase-metrics2.template.file": "hadoop-metrics2-hbase.properties-MASTER.j2"
+        },
+        "HBASE_REGIONSERVER": {
+            "site.global.hbase_component": "regionserver"
+        },
+        "HBASE_REST": {
+            "site.global.hbase_component": "rest"
+        },
+        "HBASE_THRIFT": {
+            "site.global.hbase_component": "thrift",
+            "site.global.start_command": "${@//site/global/daemon_cmd} start thrift -p ${@//site/global/thrift_port}",
+            "site.global.stop_command": "${@//site/global/daemon_cmd} stop thrift"
+        },
+        "HBASE_THRIFT2": {
+            "site.global.hbase_component": "thrift2",
+            "site.global.start_command": "${@//site/global/daemon_cmd} start thrift2 -p ${@//site/global/thrift2_port}",
+            "site.global.stop_command": "${@//site/global/daemon_cmd} stop thrift2"
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/appConfig-secured-default.json
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/appConfig-secured-default.json b/app-packages/hbase-nopkg/appConfig-secured-default.json
new file mode 100644
index 0000000..40040e3
--- /dev/null
+++ b/app-packages/hbase-nopkg/appConfig-secured-default.json
@@ -0,0 +1,120 @@
+{
+    "schema": "http://example.org/specification/v2.0.0",
+    "metadata": {
+    },
+    "global": {
+        "create.default.zookeeper.node": "true",
+        "java_home": "/usr/jdk64/jdk1.8.0_60",
+        "system_configs": "core-site,hdfs-site",
+
+        "site.global.app_user": "${USER_NAME}",
+        "site.global.app_version": "${hbase.version}",
+        "site.global.app_root": "app/packages/hbase-${hbase.version}",
+        "site.global.app_conf_dir": "app/conf",
+
+        "site.global.hbase_instance_name": "instancename",
+        "site.global.hbase_root_password": "secret",
+        "site.global.user_group": "hadoop",
+        "site.global.monitor_protocol": "http",
+        "site.global.hbase_thrift_port": "0",
+        "site.global.hbase_thrift2_port": "0",
+        "site.global.hbase_rest_port": "0",
+
+        "site.hbase-env.template.file": "hbase-env-secured.sh.j2",
+        "site.hbase-env.java64_home": "${JAVA_HOME}",
+        "site.hbase-env.conf_dir": "${@//site/global/app_conf_dir}",
+        "site.hbase-env.log_dir": "${@//site/global/app_log_dir}",
+        "site.hbase-env.pid_dir": "${@//site/global/app_pid_dir}",
+        "site.hbase-env.master_heapsize": "1024m",
+        "site.hbase-env.regionserver_heapsize": "1024m",
+        "site.hbase-env.regionserver_xmn_size": "512m",
+        "site.hbase-env.hbase_opts": "",
+        "site.hbase-env.client_jaas_config_file": "${@//site/hbase-env/conf_dir}/hbase_client_jaas.conf",
+        "site.hbase-env.master_jaas_config_file": "${@//site/hbase-env/conf_dir}/hbase_master_jaas.conf",
+        "site.hbase-env.regionserver_jaas_config_file": "${@//site/hbase-env/conf_dir}/hbase_regionserver_jaas.conf",
+
+        "site.hbase-site.hbase.rootdir": "${DEFAULT_DATA_DIR}",
+        "site.hbase-site.hbase.cluster.distributed": "true",
+        "site.hbase-site.hbase.superuser": "${USER_NAME}",
+        "site.hbase-site.hbase.tmp.dir": "${AGENT_WORK_ROOT}/work/app/tmp",
+        "site.hbase-site.hbase.local.dir": "${hbase.tmp.dir}/local",
+        "site.hbase-site.hbase.zookeeper.quorum": "${ZK_HOST}",
+        "site.hbase-site.zookeeper.znode.parent": "${DEFAULT_ZK_PATH}",
+        "site.hbase-site.hbase.regionserver.info.port": "0",
+        "site.hbase-site.hbase.master.info.port": "0",
+        "site.hbase-site.hbase.regionserver.port": "0",
+        "site.hbase-site.hbase.master.port": "0",
+        "site.hbase-site.hbase.regionserver.global.memstore.upperLimit": "0.4",
+        "site.hbase-site.hbase.regionserver.handler.count": "60",
+        "site.hbase-site.hbase.hregion.majorcompaction": "86400000",
+        "site.hbase-site.hbase.regionserver.global.memstore.lowerLimit": "0.38",
+        "site.hbase-site.hbase.hregion.memstore.block.multiplier": "2",
+        "site.hbase-site.hbase.hregion.memstore.flush.size": "134217728",
+        "site.hbase-site.hbase.hregion.memstore.mslab.enabled": "true",
+        "site.hbase-site.hbase.hregion.max.filesize": "10737418240",
+        "site.hbase-site.hbase.client.scanner.caching": "100",
+        "site.hbase-site.zookeeper.session.timeout": "30000",
+        "site.hbase-site.hbase.client.keyvalue.maxsize": "10485760",
+        "site.hbase-site.hbase.hstore.compactionThreshold": "3",
+        "site.hbase-site.hbase.hstore.flush.retries.number": "120",
+        "site.hbase-site.hbase.hstore.blockingStoreFiles": "10",
+        "site.hbase-site.hfile.block.cache.size": "0.40",
+        "site.hbase-site.hbase.zookeeper.property.clientPort": "2181",
+        "site.hbase-site.hbase.zookeeper.useMulti": "true",
+        "site.hbase-site.hbase.defaults.for.version.skip": "true",
+        "site.hbase-site.dfs.domain.socket.path": "/var/lib/hadoop-hdfs/dn_socket",
+
+        "site.hbase-site.hbase.security.authentication": "kerberos",
+        "site.core-site.hadoop.security.authentication": "kerberos",
+        "site.hbase-site.hbase.security.authorization": "true",
+        "site.hbase-site.hbase.security.access.early_out": "true",
+        "site.hbase-site.hbase.coprocessor.master.classes": "org.apache.hadoop.hbase.security.access.AccessController",
+        "site.hbase-site.hbase.coprocessor.region.classes": "org.apache.hadoop.hbase.security.token.TokenProvider,org.apache.hadoop.hbase.security.access.AccessController,org.apache.hadoop.hbase.security.access.SecureBulkLoadEndpoint",
+        "site.hbase-site.hbase.regionserver.kerberos.principal": "${USER_NAME}/${THIS_HOST}@EXAMPLE.COM",
+        "site.hbase-site.hbase.regionserver.keytab.file": "${AGENT_WORK_ROOT}/keytabs/${USER_NAME}.HBASE.service.keytab",
+        "site.hbase-site.hbase.master.kerberos.principal": "${USER_NAME}/${THIS_HOST}@EXAMPLE.COM",
+        "site.hbase-site.hbase.master.keytab.file": "${AGENT_WORK_ROOT}/keytabs/${USER_NAME}.HBASE.service.keytab",
+        "site.hbase-site.hbase.rest.kerberos.principal": "${USER_NAME}/${THIS_HOST}@EXAMPLE.COM",
+        "site.hbase-site.hbase.rest.keytab.file": "${AGENT_WORK_ROOT}/keytabs/${USER_NAME}.HBASE.service.keytab",
+        "site.hbase-site.hbase.thrift.kerberos.principal": "${USER_NAME}/${THIS_HOST}@EXAMPLE.COM",
+        "site.hbase-site.hbase.thrift.keytab.file": "${AGENT_WORK_ROOT}/keytabs/${USER_NAME}.HBASE.service.keytab",
+
+        "site.hbase-metrics2.template.file": "hadoop-metrics2-hbase.properties-RS.j2",
+        "site.hbase-metrics2.app_name": "${CLUSTER_NAME}",
+        "site.hbase-metrics2.metric_collector_host": "${NN_HOST}",
+        "site.hbase-metrics2.metric_collector_port": "6188",
+        "site.hbase-metrics2.metric_collector_lib": "",
+
+        "site.hbase-jaas.master_keytab_path": "${@//site/hbase-site/hbase.master.keytab.file}",
+        "site.hbase-jaas.master_jaas_princ": "${@//site/hbase-site/hbase.master.kerberos.principal}",
+        "site.hbase-jaas.regionserver_keytab_path": "${@//site/hbase-site/hbase.regionserver.keytab.file}",
+        "site.hbase-jaas.regionserver_jaas_princ": "${@//site/hbase-site/hbase.regionserver.kerberos.principal}"
+    },
+    "components": {
+        "slider-appmaster": {
+            "jvm.heapsize": "1024M",
+            "slider.hdfs.keytab.dir": ".slider/keytabs/hbase",
+            "slider.am.login.keytab.name": "${USER_NAME}.headless.keytab",
+            "slider.keytab.principal.name": "${USER_NAME}"
+        },
+        "HBASE_MASTER": {
+            "site.hbase-metrics2.template.file": "hadoop-metrics2-hbase.properties-MASTER.j2",
+            "site.hbase-jaas.template.file": "hbase_master_jaas.conf.j2"
+        },
+        "HBASE_REGIONSERVER": {
+            "site.hbase-jaas.template.file": "hbase_regionserver_jaas.conf.j2"
+        },
+        "HBASE_REST": {
+            "site.hbase-jaas.template.file": "hbase_regionserver_jaas.conf.j2"
+        },
+        "HBASE_THRIFT": {
+            "site.hbase-jaas.template.file": "hbase_regionserver_jaas.conf.j2"
+        },
+        "HBASE_THRIFT2": {
+            "site.hbase-jaas.template.file": "hbase_regionserver_jaas.conf.j2"
+        },
+        "HBASE_CLIENT": {
+            "site.hbase-jaas.template.file": "hbase_client_jaas.conf.j2"
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/metainfo-secured.xml
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/metainfo-secured.xml b/app-packages/hbase-nopkg/metainfo-secured.xml
new file mode 100644
index 0000000..11c7d0a
--- /dev/null
+++ b/app-packages/hbase-nopkg/metainfo-secured.xml
@@ -0,0 +1,235 @@
+<?xml version="1.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.
+-->
+<metainfo>
+  <schemaVersion>2.0</schemaVersion>
+  <application>
+    <name>HBASE</name>
+    <comment>
+      Apache HBase is the Hadoop database, a distributed, scalable, big data store.
+      Requirements:
+      1. Ensure parent dir for path (hbase-site/hbase.rootdir) is accessible to the App owner.
+      2. Ensure ZK root (hbase-site/zookeeper.znode.parent) is unique for the App instance.
+    </comment>
+    <version>${pkg.version}</version>
+    <type>YARN-APP</type>
+    <minHadoopVersion>2.1.0</minHadoopVersion>
+    <exportedConfigs>hbase-site</exportedConfigs>
+    <exportGroups>
+      <exportGroup>
+        <name>QuickLinks</name>
+        <exports>
+          <export>
+            <name>org.apache.slider.jmx</name>
+            <value>http://${HBASE_MASTER_HOST}:${site.hbase-site.hbase.master.info.port}/jmx</value>
+          </export>
+          <export>
+            <name>org.apache.slider.monitor</name>
+            <value>http://${HBASE_MASTER_HOST}:${site.hbase-site.hbase.master.info.port}/master-status</value>
+          </export>
+          <export>
+            <name>org.apache.slider.hbase.rest</name>
+            <value>http://${HBASE_REST_HOST}:${site.global.hbase_rest_port}</value>
+          </export>
+          <export>
+            <name>org.apache.slider.hbase.thrift2</name>
+            <value>http://${HBASE_THRIFT2_HOST}:${site.global.hbase_thrift2_port}</value>
+          </export>
+          <export>
+            <name>org.apache.slider.hbase.thrift</name>
+            <value>http://${HBASE_THRIFT_HOST}:${site.global.hbase_thrift_port}</value>
+          </export>
+          <export>
+            <name>org.apache.slider.metrics</name>
+            <value>http://${site.global.metric_collector_host}:${site.global.metric_collector_port}/ws/v1/timeline/metrics</value>
+          </export>
+          <export>
+            <name>org.apache.slider.hbase.zk-path</name>
+            <value>${site.hbase-site.hbase.zookeeper.quorum}:${site.hbase-site.zookeeper.znode.parent}</value>
+          </export>
+        </exports>
+      </exportGroup>
+    </exportGroups>
+    <commandOrders>
+      <commandOrder>
+        <command>HBASE_REGIONSERVER-START</command>
+        <requires>HBASE_MASTER-STARTED</requires>
+      </commandOrder>
+      <commandOrder>
+        <command>HBASE_MASTER-START</command>
+        <requires>HBASE_REST-INSTALLED</requires>
+        <requires>HBASE_THRIFT-INSTALLED</requires>
+        <requires>HBASE_THRIFT2-INSTALLED</requires>
+      </commandOrder>
+    </commandOrders>
+    <components>
+      <component>
+        <name>HBASE_MASTER</name>
+        <category>MASTER</category>
+        <minInstanceCount>1</minInstanceCount>
+        <appExports>QuickLinks-org.apache.slider.jmx,QuickLinks-org.apache.slider.monitor,QuickLinks-org.apache.slider.metrics,QuickLinks-org.apache.slider.hbase.zk-path</appExports>
+        <componentExports>
+          <componentExport>
+            <name>org.apache.slider.jmx</name>
+            <value>${THIS_HOST}:${site.hbase-site.hbase.master.info.port}/jmx</value>
+          </componentExport>
+          <componentExport>
+            <name>org.apache.slider.monitor</name>
+            <value>${THIS_HOST}:${site.hbase-site.hbase.master.info.port}/master-status</value>
+          </componentExport>
+          <componentExport>
+            <name>org.apache.slider.hbase.zk-path</name>
+            <value>${site.hbase-site.hbase.zookeeper.quorum}:${site.hbase-site.zookeeper.znode.parent}</value>
+          </componentExport>
+        </componentExports>
+        <commandScript>
+          <script>scripts/hbase_master.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+        <configFile>
+          <type>template</type>
+          <fileName>app/conf/hbase_master_jaas.conf</fileName>
+          <dictionaryName>hbase-jaas</dictionaryName>
+        </configFile>
+      </component>
+
+      <component>
+        <name>HBASE_REGIONSERVER</name>
+        <category>SLAVE</category>
+        <minInstanceCount>1</minInstanceCount>
+        <commandScript>
+          <script>scripts/hbase_regionserver.py</script>
+          <scriptType>PYTHON</scriptType>
+        </commandScript>
+        <configFile>
+          <type>template</type>
+          <fileName>app/conf/hbase_regionserver_jaas.conf</fileName>
+          <dictionaryName>hbase-jaas</dictionaryName>
+        </configFile>
+      </component>
+
+      <component>
+        <name>HBASE_REST</name>
+        <category>MASTER</category>
+        <appExports>QuickLinks-org.apache.slider.hbase.rest</appExports>
+        <commandScript>
+          <script>scripts/hbase_rest.py</script>
+          <scriptType>PYTHON</scriptType>
+        </commandScript>
+        <configFile>
+          <type>template</type>
+          <fileName>app/conf/hbase_rest_jaas.conf</fileName>
+          <dictionaryName>hbase-jaas</dictionaryName>
+        </configFile>
+      </component>
+
+      <component>
+        <name>HBASE_THRIFT</name>
+        <category>MASTER</category>
+        <appExports>QuickLinks-org.apache.slider.hbase.thrift</appExports>
+        <commandScript>
+          <script>scripts/hbase_thrift.py</script>
+          <scriptType>PYTHON</scriptType>
+        </commandScript>
+        <configFile>
+          <type>template</type>
+          <fileName>app/conf/hbase_thrift2_jaas.conf</fileName>
+          <dictionaryName>hbase-jaas</dictionaryName>
+        </configFile>
+      </component>
+
+      <component>
+        <name>HBASE_THRIFT2</name>
+        <category>MASTER</category>
+        <minInstanceCount>0</minInstanceCount>
+        <appExports>QuickLinks-org.apache.slider.hbase.thrift2</appExports>
+        <commandScript>
+          <script>scripts/hbase_thrift2.py</script>
+          <scriptType>PYTHON</scriptType>
+        </commandScript>
+        <configFile>
+          <type>template</type>
+          <fileName>app/conf/hbase_thrift_jaas.conf</fileName>
+          <dictionaryName>hbase-jaas</dictionaryName>
+        </configFile>
+      </component>
+
+      <component>
+        <name>HBASE_CLIENT</name>
+        <category>CLIENT</category>
+        <commandScript>
+          <script>scripts/hbase_client.py</script>
+          <scriptType>PYTHON</scriptType>
+        </commandScript>
+        <configFile>
+          <type>template</type>
+          <fileName>app/conf/hbase_client_jaas.conf</fileName>
+          <dictionaryName>hbase-jaas</dictionaryName>
+        </configFile>
+      </component>
+    </components>
+
+    <osSpecifics>
+      <osSpecific>
+        <osType>any</osType>
+        <packages>
+          <package>
+            <type>tarball</type>
+            <name>files/hbase-${pkg.version}.tar.gz</name>
+          </package>
+        </packages>
+      </osSpecific>
+    </osSpecifics>
+
+    <packages>
+      <package>
+        <type>tarball</type>
+        <name>hbase-${pkg.version}-bin.tar.gz</name>
+      </package>
+    </packages>
+
+    <configFiles>
+      <configFile>
+        <type>xml</type>
+        <fileName>app/conf/hbase-site.xml</fileName>
+        <dictionaryName>hbase-site</dictionaryName>
+      </configFile>
+      <configFile>
+        <type>template</type>
+        <fileName>app/conf/hbase-env.sh</fileName>
+        <dictionaryName>hbase-env</dictionaryName>
+      </configFile>
+      <configFile>
+        <type>template</type>
+        <fileName>app/conf/log4j.properties</fileName>
+        <dictionaryName>hbase-log4j</dictionaryName>
+      </configFile>
+      <configFile>
+        <type>xml</type>
+        <fileName>app/conf/hbase-policy.xml</fileName>
+        <dictionaryName>hbase-policy</dictionaryName>
+      </configFile>
+      <configFile>
+        <type>template</type>
+        <fileName>app/conf/hadoop-metrics2-hbase.properties</fileName>
+        <dictionaryName>hbase-metrics2</dictionaryName>
+      </configFile>
+    </configFiles>
+
+  </application>
+</metainfo>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/metainfo.xml
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/metainfo.xml b/app-packages/hbase-nopkg/metainfo.xml
new file mode 100644
index 0000000..9abc9dc
--- /dev/null
+++ b/app-packages/hbase-nopkg/metainfo.xml
@@ -0,0 +1,229 @@
+<?xml version="1.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.
+-->
+<metainfo>
+  <schemaVersion>2.0</schemaVersion>
+  <application>
+    <name>HBASE</name>
+    <comment>
+      Apache HBase is the Hadoop database, a distributed, scalable, big data store.
+      Requirements:
+      1. Ensure parent dir for path (hbase-site/hbase.rootdir) is accessible to the App owner.
+      2. Ensure ZK root (hbase-site/zookeeper.znode.parent) is unique for the App instance.
+    </comment>
+    <version>${pkg.version}</version>
+    <type>YARN-APP</type>
+    <minHadoopVersion>2.1.0</minHadoopVersion>
+    <exportedConfigs>hbase-site</exportedConfigs>
+    <exportGroups>
+      <exportGroup>
+        <name>QuickLinks</name>
+        <exports>
+          <export>
+            <name>org.apache.slider.jmx</name>
+            <value>http://${HBASE_MASTER_HOST}:${site.hbase-site.hbase.master.info.port}/jmx</value>
+          </export>
+          <export>
+            <name>org.apache.slider.monitor</name>
+            <value>http://${HBASE_MASTER_HOST}:${site.hbase-site.hbase.master.info.port}/master-status</value>
+          </export>
+          <export>
+            <name>org.apache.slider.hbase.rest</name>
+            <value>http://${HBASE_REST_HOST}:${site.global.hbase_rest_port}</value>
+          </export>
+          <export>
+            <name>org.apache.slider.hbase.thrift2</name>
+            <value>http://${HBASE_THRIFT2_HOST}:${site.global.hbase_thrift2_port}</value>
+          </export>
+          <export>
+            <name>org.apache.slider.hbase.thrift</name>
+            <value>http://${HBASE_THRIFT_HOST}:${site.global.hbase_thrift_port}</value>
+          </export>
+          <export>
+            <name>org.apache.slider.metrics</name>
+            <value>http://${site.global.metric_collector_host}:${site.global.metric_collector_port}/ws/v1/timeline/metrics</value>
+          </export>
+          <export>
+            <name>org.apache.slider.hbase.zk-path</name>
+            <value>${site.hbase-site.hbase.zookeeper.quorum}:${site.hbase-site.zookeeper.znode.parent}</value>
+          </export>
+        </exports>
+      </exportGroup>
+    </exportGroups>
+    <commandOrders>
+      <commandOrder>
+        <command>HBASE_REGIONSERVER-START</command>
+        <requires>HBASE_MASTER-STARTED</requires>
+      </commandOrder>
+      <commandOrder>
+        <command>HBASE_MASTER-START</command>
+        <requires>HBASE_REST-INSTALLED</requires>
+        <requires>HBASE_THRIFT-INSTALLED</requires>
+        <requires>HBASE_THRIFT2-INSTALLED</requires>
+      </commandOrder>
+    </commandOrders>
+    <components>
+      <component>
+        <name>HBASE_MASTER</name>
+        <category>MASTER</category>
+        <minInstanceCount>1</minInstanceCount>
+        <appExports>QuickLinks-org.apache.slider.jmx,QuickLinks-org.apache.slider.monitor,QuickLinks-org.apache.slider.metrics,QuickLinks-org.apache.slider.hbase.zk-path</appExports>
+        <componentExports>
+          <componentExport>
+            <name>org.apache.slider.jmx</name>
+            <value>${THIS_HOST}:${site.hbase-site.hbase.master.info.port}/jmx</value>
+          </componentExport>
+          <componentExport>
+            <name>org.apache.slider.monitor</name>
+            <value>${THIS_HOST}:${site.hbase-site.hbase.master.info.port}/master-status</value>
+          </componentExport>
+          <componentExport>
+            <name>org.apache.slider.hbase.zk-path</name>
+            <value>${site.hbase-site.hbase.zookeeper.quorum}:${site.hbase-site.zookeeper.znode.parent}</value>
+          </componentExport>
+        </componentExports>
+        <commands>
+          <command>
+            <exec>{$conf:@//site/global/daemon_cmd} start master</exec>
+          </command>
+          <command>
+            <exec>{$conf:@//site/global/daemon_cmd} stop master</exec>
+            <name>STOP</name>
+          </command>
+        </commands>
+      </component>
+
+      <component>
+        <name>HBASE_REGIONSERVER</name>
+        <category>SLAVE</category>
+        <minInstanceCount>1</minInstanceCount>
+        <commands>
+          <command>
+            <exec>{$conf:@//site/global/daemon_cmd} start regionserver</exec>
+          </command>
+          <command>
+            <exec>{$conf:@//site/global/daemon_cmd} stop regionserver</exec>
+            <name>STOP</name>
+          </command>
+        </commands>
+      </component>
+
+      <component>
+        <name>HBASE_REST</name>
+        <category>MASTER</category>
+        <appExports>QuickLinks-org.apache.slider.hbase.rest</appExports>
+        <commands>
+          <command>
+            <exec>{$conf:@//site/global/daemon_cmd} start rest -p {$conf:@//site/global/rest_port}</exec>
+          </command>
+          <command>
+            <exec>{$conf:@//site/global/daemon_cmd} stop rest</exec>
+            <name>STOP</name>
+          </command>
+        </commands>
+      </component>
+
+      <component>
+        <name>HBASE_THRIFT</name>
+        <category>MASTER</category>
+        <appExports>QuickLinks-org.apache.slider.hbase.thrift</appExports>
+        <commands>
+          <command>
+            <exec>{$conf:@//site/global/daemon_cmd} start thrift -p {$conf:@//site/global/thrift_port}</exec>
+          </command>
+          <command>
+            <exec>{$conf:@//site/global/daemon_cmd} stop thrift</exec>
+            <name>STOP</name>
+          </command>
+        </commands>
+      </component>
+
+      <component>
+        <name>HBASE_THRIFT2</name>
+        <category>MASTER</category>
+        <minInstanceCount>0</minInstanceCount>
+        <appExports>QuickLinks-org.apache.slider.hbase.thrift2</appExports>
+        <commands>
+          <command>
+            <exec>{$conf:@//site/global/daemon_cmd} start thrift2 -p {$conf:@//site/global/thrift_port}</exec>
+          </command>
+          <command>
+            <exec>{$conf:@//site/global/daemon_cmd} stop thrift2</exec>
+            <name>STOP</name>
+          </command>
+        </commands>
+      </component>
+
+      <component>
+        <name>HBASE_CLIENT</name>
+        <category>CLIENT</category>
+        <commandScript>
+          <script>scripts/hbase_client.py</script>
+          <scriptType>PYTHON</scriptType>
+        </commandScript>
+      </component>
+    </components>
+
+    <osSpecifics>
+      <osSpecific>
+        <osType>any</osType>
+        <packages>
+          <package>
+            <type>tarball</type>
+            <name>files/hbase-${pkg.version}.tar.gz</name>
+          </package>
+        </packages>
+      </osSpecific>
+    </osSpecifics>
+
+    <packages>
+      <package>
+        <type>archive</type>
+        <name>hbase-${pkg.version}-bin.tar.gz</name>
+      </package>
+    </packages>
+
+    <configFiles>
+      <configFile>
+        <type>xml</type>
+        <fileName>app/conf/hbase-site.xml</fileName>
+        <dictionaryName>hbase-site</dictionaryName>
+      </configFile>
+      <configFile>
+        <type>template</type>
+        <fileName>app/conf/hbase-env.sh</fileName>
+        <dictionaryName>hbase-env</dictionaryName>
+      </configFile>
+      <configFile>
+        <type>template</type>
+        <fileName>app/conf/log4j.properties</fileName>
+        <dictionaryName>hbase-log4j</dictionaryName>
+      </configFile>
+      <configFile>
+        <type>xml</type>
+        <fileName>app/conf/hbase-policy.xml</fileName>
+        <dictionaryName>hbase-policy</dictionaryName>
+      </configFile>
+      <configFile>
+        <type>template</type>
+        <fileName>app/conf/hadoop-metrics2-hbase.properties</fileName>
+        <dictionaryName>hbase-metrics2</dictionaryName>
+      </configFile>
+    </configFiles>
+
+  </application>
+</metainfo>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/pom.xml b/app-packages/hbase-nopkg/pom.xml
new file mode 100644
index 0000000..fcba6f2
--- /dev/null
+++ b/app-packages/hbase-nopkg/pom.xml
@@ -0,0 +1,92 @@
+<?xml version="1.0"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <!--
+   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.
+-->
+  <parent>
+    <groupId>org.apache.slider.packages</groupId>
+    <artifactId>slider-app-packages</artifactId>
+    <version>0.91.0-incubating-SNAPSHOT</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>slider-hbase-resources</artifactId>
+  <packaging>pom</packaging>
+  <name>Slider HBase Resources</name>
+  <description>Slider HBase Resources</description>
+  <properties>
+    <work.dir>package-tmp</work.dir>
+    <app.package.name>${project.artifactId}-${pkg.version}</app.package.name>
+  </properties>
+
+  <profiles>
+    <profile>
+      <id>hbase-resources</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <version>${maven-antrun-plugin.version}</version>
+            <executions>
+              <execution>
+                <id>copy</id>
+                <phase>validate</phase>
+                <configuration>
+                  <target name="copy and rename file">
+                    <copy file="${pkg.src}/${pkg.name}" tofile="${project.build.directory}/${pkg.name}" />
+                  </target>
+                </configuration>
+                <goals>
+                  <goal>run</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-assembly-plugin</artifactId>
+            <version>${maven-assembly-plugin.version}</version>
+            <configuration>
+              <tarLongFileMode>gnu</tarLongFileMode>
+              <descriptor>src/assembly/hbase.xml</descriptor>
+              <appendAssemblyId>false</appendAssemblyId>
+              <finalName>${app.package.name}</finalName>
+            </configuration>
+            <executions>
+              <execution>
+                <id>build-tarball</id>
+                <phase>package</phase>
+                <goals>
+                  <goal>single</goal>
+                </goals>
+              </execution>
+            </executions>
+          </plugin>
+
+        </plugins>
+      </build>
+    </profile>
+  </profiles>
+
+  <build>
+  </build>
+
+  <dependencies>
+  </dependencies>
+
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/resources-default.json
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/resources-default.json b/app-packages/hbase-nopkg/resources-default.json
new file mode 100644
index 0000000..de3b279
--- /dev/null
+++ b/app-packages/hbase-nopkg/resources-default.json
@@ -0,0 +1,47 @@
+{
+  "schema": "http://example.org/specification/v2.0.0",
+  "metadata": {
+  },
+  "global": {
+    "yarn.log.include.patterns": "",
+    "yarn.log.exclude.patterns": "",
+    "yarn.component.instances": "1",
+    "yarn.vcores": "1"
+  },
+  "components": {
+    "slider-appmaster": {
+      "yarn.memory": "1024"
+    },
+    "HBASE_MASTER": {
+      "yarn.role.priority": "1",
+      "yarn.component.instances": "1",
+      "yarn.placement.escalate.seconds": "10",
+      "yarn.memory": "1500"
+    },
+    "HBASE_REGIONSERVER": {
+      "yarn.role.priority": "2",
+      "yarn.component.instances": "1",
+      "yarn.memory": "1500",
+      "yarn.container.failure.threshold": "15",
+      "yarn.placement.escalate.seconds": "60"
+    },
+    "HBASE_REST": {
+      "yarn.role.priority": "3",
+      "yarn.component.instances": "1",
+      "yarn.component.placement.policy": "1",
+      "yarn.memory": "556"
+    },
+    "HBASE_THRIFT": {
+      "yarn.role.priority": "4",
+      "yarn.component.instances": "1",
+      "yarn.component.placement.policy": "1",
+      "yarn.memory": "556"
+    },
+    "HBASE_THRIFT2": {
+      "yarn.role.priority": "5",
+      "yarn.component.instances": "1",
+      "yarn.component.placement.policy": "1",
+      "yarn.memory": "556"
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-GANGLIA-MASTER.j2
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-GANGLIA-MASTER.j2 b/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-GANGLIA-MASTER.j2
new file mode 100644
index 0000000..3ed7af2
--- /dev/null
+++ b/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-GANGLIA-MASTER.j2
@@ -0,0 +1,62 @@
+# 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.
+
+# See http://wiki.apache.org/hadoop/GangliaMetrics
+#
+# Make sure you know whether you are using ganglia 3.0 or 3.1.
+# If 3.1, you will have to patch your hadoop instance with HADOOP-4675
+# And, yes, this file is named hadoop-metrics.properties rather than
+# hbase-metrics.properties because we're leveraging the hadoop metrics
+# package and hadoop-metrics.properties is an hardcoded-name, at least
+# for the moment.
+#
+# See also http://hadoop.apache.org/hbase/docs/current/metrics.html
+
+# HBase-specific configuration to reset long-running stats (e.g. compactions)
+# If this variable is left out, then the default is no expiration.
+hbase.extendedperiod = 3600
+
+# Configuration of the "hbase" context for ganglia
+# Pick one: Ganglia 3.0 (former) or Ganglia 3.1 (latter)
+# hbase.class=org.apache.hadoop.metrics.ganglia.GangliaContext
+hbase.class=org.apache.hadoop.metrics.ganglia.GangliaContext31
+hbase.period=10
+hbase.servers={{metric_collector_host}}:{{metric_collector_port}}
+
+# Configuration of the "jvm" context for ganglia
+# Pick one: Ganglia 3.0 (former) or Ganglia 3.1 (latter)
+# jvm.class=org.apache.hadoop.metrics.ganglia.GangliaContext
+jvm.class=org.apache.hadoop.metrics.ganglia.GangliaContext31
+jvm.period=10
+jvm.servers={{metric_collector_host}}:{{metric_collector_port}}
+
+# Configuration of the "rpc" context for ganglia
+# Pick one: Ganglia 3.0 (former) or Ganglia 3.1 (latter)
+# rpc.class=org.apache.hadoop.metrics.ganglia.GangliaContext
+rpc.class=org.apache.hadoop.metrics.ganglia.GangliaContext31
+rpc.period=10
+rpc.servers={{metric_collector_host}}:{{metric_collector_port}}
+
+#Ganglia following hadoop example
+hbase.sink.ganglia.class=org.apache.hadoop.metrics2.sink.ganglia.GangliaSink31
+hbase.sink.ganglia.period=10
+
+# default for supportsparse is false
+*.sink.ganglia.supportsparse=true
+
+.sink.ganglia.slope=jvm.metrics.gcCount=zero,jvm.metrics.memHeapUsedM=both
+.sink.ganglia.dmax=jvm.metrics.threadsBlocked=70,jvm.metrics.memHeapUsedM=40
+
+hbase.sink.ganglia.servers={{metric_collector_host}}:{{metric_collector_port}}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-GANGLIA-RS.j2
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-GANGLIA-RS.j2 b/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-GANGLIA-RS.j2
new file mode 100644
index 0000000..3ed7af2
--- /dev/null
+++ b/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-GANGLIA-RS.j2
@@ -0,0 +1,62 @@
+# 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.
+
+# See http://wiki.apache.org/hadoop/GangliaMetrics
+#
+# Make sure you know whether you are using ganglia 3.0 or 3.1.
+# If 3.1, you will have to patch your hadoop instance with HADOOP-4675
+# And, yes, this file is named hadoop-metrics.properties rather than
+# hbase-metrics.properties because we're leveraging the hadoop metrics
+# package and hadoop-metrics.properties is an hardcoded-name, at least
+# for the moment.
+#
+# See also http://hadoop.apache.org/hbase/docs/current/metrics.html
+
+# HBase-specific configuration to reset long-running stats (e.g. compactions)
+# If this variable is left out, then the default is no expiration.
+hbase.extendedperiod = 3600
+
+# Configuration of the "hbase" context for ganglia
+# Pick one: Ganglia 3.0 (former) or Ganglia 3.1 (latter)
+# hbase.class=org.apache.hadoop.metrics.ganglia.GangliaContext
+hbase.class=org.apache.hadoop.metrics.ganglia.GangliaContext31
+hbase.period=10
+hbase.servers={{metric_collector_host}}:{{metric_collector_port}}
+
+# Configuration of the "jvm" context for ganglia
+# Pick one: Ganglia 3.0 (former) or Ganglia 3.1 (latter)
+# jvm.class=org.apache.hadoop.metrics.ganglia.GangliaContext
+jvm.class=org.apache.hadoop.metrics.ganglia.GangliaContext31
+jvm.period=10
+jvm.servers={{metric_collector_host}}:{{metric_collector_port}}
+
+# Configuration of the "rpc" context for ganglia
+# Pick one: Ganglia 3.0 (former) or Ganglia 3.1 (latter)
+# rpc.class=org.apache.hadoop.metrics.ganglia.GangliaContext
+rpc.class=org.apache.hadoop.metrics.ganglia.GangliaContext31
+rpc.period=10
+rpc.servers={{metric_collector_host}}:{{metric_collector_port}}
+
+#Ganglia following hadoop example
+hbase.sink.ganglia.class=org.apache.hadoop.metrics2.sink.ganglia.GangliaSink31
+hbase.sink.ganglia.period=10
+
+# default for supportsparse is false
+*.sink.ganglia.supportsparse=true
+
+.sink.ganglia.slope=jvm.metrics.gcCount=zero,jvm.metrics.memHeapUsedM=both
+.sink.ganglia.dmax=jvm.metrics.threadsBlocked=70,jvm.metrics.memHeapUsedM=40
+
+hbase.sink.ganglia.servers={{metric_collector_host}}:{{metric_collector_port}}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-MASTER.j2
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-MASTER.j2 b/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-MASTER.j2
new file mode 100644
index 0000000..5473ff5
--- /dev/null
+++ b/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-MASTER.j2
@@ -0,0 +1,47 @@
+# 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.
+
+# See http://wiki.apache.org/hadoop/GangliaMetrics
+#
+# Make sure you know whether you are using ganglia 3.0 or 3.1.
+# If 3.1, you will have to patch your hadoop instance with HADOOP-4675
+# And, yes, this file is named hadoop-metrics.properties rather than
+# hbase-metrics.properties because we're leveraging the hadoop metrics
+# package and hadoop-metrics.properties is an hardcoded-name, at least
+# for the moment.
+#
+# See also http://hadoop.apache.org/hbase/docs/current/metrics.html
+
+# HBase-specific configuration to reset long-running stats (e.g. compactions)
+# If this variable is left out, then the default is no expiration.
+hbase.extendedperiod = 3600
+
+*.timeline.plugin.urls={{metric_collector_lib}}
+hbase.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+hbase.period=10
+hbase.collector={{metric_collector_host}}:{{metric_collector_port}}
+
+jvm.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+jvm.period=10
+jvm.collector={{metric_collector_host}}:{{metric_collector_port}}
+
+rpc.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+rpc.period=10
+rpc.collector={{metric_collector_host}}:{{metric_collector_port}}
+
+hbase.sink.timeline.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+hbase.sink.timeline.period=10
+hbase.sink.timeline.collector={{metric_collector_host}}:{{metric_collector_port}}
+hbase.sink.timeline.serviceName={{app_name}}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-RS.j2
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-RS.j2 b/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-RS.j2
new file mode 100644
index 0000000..5473ff5
--- /dev/null
+++ b/app-packages/hbase-nopkg/resources/hadoop-metrics2-hbase.properties-RS.j2
@@ -0,0 +1,47 @@
+# 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.
+
+# See http://wiki.apache.org/hadoop/GangliaMetrics
+#
+# Make sure you know whether you are using ganglia 3.0 or 3.1.
+# If 3.1, you will have to patch your hadoop instance with HADOOP-4675
+# And, yes, this file is named hadoop-metrics.properties rather than
+# hbase-metrics.properties because we're leveraging the hadoop metrics
+# package and hadoop-metrics.properties is an hardcoded-name, at least
+# for the moment.
+#
+# See also http://hadoop.apache.org/hbase/docs/current/metrics.html
+
+# HBase-specific configuration to reset long-running stats (e.g. compactions)
+# If this variable is left out, then the default is no expiration.
+hbase.extendedperiod = 3600
+
+*.timeline.plugin.urls={{metric_collector_lib}}
+hbase.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+hbase.period=10
+hbase.collector={{metric_collector_host}}:{{metric_collector_port}}
+
+jvm.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+jvm.period=10
+jvm.collector={{metric_collector_host}}:{{metric_collector_port}}
+
+rpc.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+rpc.period=10
+rpc.collector={{metric_collector_host}}:{{metric_collector_port}}
+
+hbase.sink.timeline.class=org.apache.hadoop.metrics2.sink.timeline.HadoopTimelineMetricsSink
+hbase.sink.timeline.period=10
+hbase.sink.timeline.collector={{metric_collector_host}}:{{metric_collector_port}}
+hbase.sink.timeline.serviceName={{app_name}}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/resources/hbase-env-secured.sh.j2
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/resources/hbase-env-secured.sh.j2 b/app-packages/hbase-nopkg/resources/hbase-env-secured.sh.j2
new file mode 100644
index 0000000..9ee901d
--- /dev/null
+++ b/app-packages/hbase-nopkg/resources/hbase-env-secured.sh.j2
@@ -0,0 +1,63 @@
+# Set environment variables here.
+
+# The java implementation to use. Java 1.6 required.
+export JAVA_HOME={{java64_home}}
+
+# HBase Configuration directory
+export HBASE_CONF_DIR=${HBASE_CONF_DIR:-{{conf_dir}}}
+
+# Extra Java CLASSPATH elements. Optional.
+export HBASE_CLASSPATH=${HBASE_CLASSPATH}
+
+# The maximum amount of heap to use, in MB. Default is 1000.
+# export HBASE_HEAPSIZE=1000
+
+# Extra Java runtime options.
+# Below are what we set by default. May only work with SUN JVM.
+# For more on why as well as other possible settings,
+# see http://wiki.apache.org/hadoop/PerformanceTuning
+export HBASE_OPTS="-XX:+UseConcMarkSweepGC -XX:ErrorFile={{log_dir}}/hs_err_pid%p.log"
+export SERVER_GC_OPTS="-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -Xloggc:{{log_dir}}/gc.log-`date +'%Y%m%d%H%M'`"
+# Uncomment below to enable java garbage collection logging.
+# export HBASE_OPTS="$HBASE_OPTS -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -Xloggc:$HBASE_HOME/logs/gc-hbase.log"
+
+# Uncomment and adjust to enable JMX exporting
+# See jmxremote.password and jmxremote.access in $JRE_HOME/lib/management to configure remote password access.
+# More details at: http://java.sun.com/javase/6/docs/technotes/guides/management/agent.html
+#
+# export HBASE_JMX_BASE="-Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false"
+export HBASE_MASTER_OPTS="-Xmx{{master_heapsize}}"
+export HBASE_REGIONSERVER_OPTS="-Xmn{{regionserver_xmn_size}} -XX:CMSInitiatingOccupancyFraction=70  -Xms{{regionserver_heapsize}} -Xmx{{regionserver_heapsize}}"
+# export HBASE_THRIFT_OPTS="$HBASE_JMX_BASE -Dcom.sun.management.jmxremote.port=10103"
+# export HBASE_ZOOKEEPER_OPTS="$HBASE_JMX_BASE -Dcom.sun.management.jmxremote.port=10104"
+
+# File naming hosts on which HRegionServers will run. $HBASE_HOME/conf/regionservers by default.
+export HBASE_REGIONSERVERS=${HBASE_CONF_DIR}/regionservers
+
+# Extra ssh options. Empty by default.
+# export HBASE_SSH_OPTS="-o ConnectTimeout=1 -o SendEnv=HBASE_CONF_DIR"
+
+# Where log files are stored. $HBASE_HOME/logs by default.
+export HBASE_LOG_DIR={{log_dir}}
+
+# A string representing this instance of hbase. $USER by default.
+# export HBASE_IDENT_STRING=$USER
+
+# The scheduling priority for daemon processes. See 'man nice'.
+# export HBASE_NICENESS=10
+
+# The directory where pid files are stored. /tmp by default.
+export HBASE_PID_DIR={{pid_dir}}
+
+# Seconds to sleep between slave commands. Unset by default. This
+# can be useful in large clusters, where, e.g., slave rsyncs can
+# otherwise arrive faster than the master can service them.
+# export HBASE_SLAVE_SLEEP=0.1
+
+# Tell HBase whether it should manage it's own instance of Zookeeper or not.
+export HBASE_MANAGES_ZK=false
+export HBASE_OPTS="$HBASE_OPTS {{hbase_opts}}"
+
+export HBASE_OPTS="$HBASE_OPTS -Djava.security.auth.login.config={{client_jaas_config_file}}"
+export HBASE_MASTER_OPTS="$HBASE_MASTER_OPTS -Djava.security.auth.login.config={{master_jaas_config_file}}"
+export HBASE_REGIONSERVER_OPTS="$HBASE_REGIONSERVER_OPTS -Djava.security.auth.login.config={{regionserver_jaas_config_file}}"

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/resources/hbase-env.sh.j2
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/resources/hbase-env.sh.j2 b/app-packages/hbase-nopkg/resources/hbase-env.sh.j2
new file mode 100644
index 0000000..3ed354f
--- /dev/null
+++ b/app-packages/hbase-nopkg/resources/hbase-env.sh.j2
@@ -0,0 +1,59 @@
+# Set environment variables here.
+
+# The java implementation to use. Java 1.6 required.
+export JAVA_HOME={{java64_home}}
+
+# HBase Configuration directory
+export HBASE_CONF_DIR=${HBASE_CONF_DIR:-{{conf_dir}}}
+
+# Extra Java CLASSPATH elements. Optional.
+export HBASE_CLASSPATH=${HBASE_CLASSPATH}
+
+# The maximum amount of heap to use, in MB. Default is 1000.
+# export HBASE_HEAPSIZE=1000
+
+# Extra Java runtime options.
+# Below are what we set by default. May only work with SUN JVM.
+# For more on why as well as other possible settings,
+# see http://wiki.apache.org/hadoop/PerformanceTuning
+export HBASE_OPTS="-XX:+UseConcMarkSweepGC -XX:ErrorFile={{log_dir}}/hs_err_pid%p.log"
+export SERVER_GC_OPTS="-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -Xloggc:{{log_dir}}/gc.log-`date +'%Y%m%d%H%M'`"
+# Uncomment below to enable java garbage collection logging.
+# export HBASE_OPTS="$HBASE_OPTS -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCDateStamps -Xloggc:$HBASE_HOME/logs/gc-hbase.log"
+
+# Uncomment and adjust to enable JMX exporting
+# See jmxremote.password and jmxremote.access in $JRE_HOME/lib/management to configure remote password access.
+# More details at: http://java.sun.com/javase/6/docs/technotes/guides/management/agent.html
+#
+# export HBASE_JMX_BASE="-Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false"
+export HBASE_MASTER_OPTS="-Xmx{{master_heapsize}}"
+export HBASE_REGIONSERVER_OPTS="-Xmn{{regionserver_xmn_size}} -XX:CMSInitiatingOccupancyFraction=70  -Xms{{regionserver_heapsize}} -Xmx{{regionserver_heapsize}}"
+# export HBASE_THRIFT_OPTS="$HBASE_JMX_BASE -Dcom.sun.management.jmxremote.port=10103"
+# export HBASE_ZOOKEEPER_OPTS="$HBASE_JMX_BASE -Dcom.sun.management.jmxremote.port=10104"
+
+# File naming hosts on which HRegionServers will run. $HBASE_HOME/conf/regionservers by default.
+export HBASE_REGIONSERVERS=${HBASE_CONF_DIR}/regionservers
+
+# Extra ssh options. Empty by default.
+# export HBASE_SSH_OPTS="-o ConnectTimeout=1 -o SendEnv=HBASE_CONF_DIR"
+
+# Where log files are stored. $HBASE_HOME/logs by default.
+export HBASE_LOG_DIR={{log_dir}}
+
+# A string representing this instance of hbase. $USER by default.
+# export HBASE_IDENT_STRING=$USER
+
+# The scheduling priority for daemon processes. See 'man nice'.
+# export HBASE_NICENESS=10
+
+# The directory where pid files are stored. /tmp by default.
+export HBASE_PID_DIR={{pid_dir}}
+
+# Seconds to sleep between slave commands. Unset by default. This
+# can be useful in large clusters, where, e.g., slave rsyncs can
+# otherwise arrive faster than the master can service them.
+# export HBASE_SLAVE_SLEEP=0.1
+
+# Tell HBase whether it should manage it's own instance of Zookeeper or not.
+export HBASE_MANAGES_ZK=false
+export HBASE_OPTS="$HBASE_OPTS {{hbase_opts}}"

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/resources/hbase-log4j.properties.j2
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/resources/hbase-log4j.properties.j2 b/app-packages/hbase-nopkg/resources/hbase-log4j.properties.j2
new file mode 100644
index 0000000..e0a7e22
--- /dev/null
+++ b/app-packages/hbase-nopkg/resources/hbase-log4j.properties.j2
@@ -0,0 +1,110 @@
+# 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.
+
+
+# Define some default values that can be overridden by system properties
+hbase.root.logger=INFO,console
+hbase.security.logger=INFO,console
+hbase.log.dir=.
+hbase.log.file=hbase.log
+
+# Define the root logger to the system property "hbase.root.logger".
+log4j.rootLogger=${hbase.root.logger}
+
+# Logging Threshold
+log4j.threshold=ALL
+
+#
+# Daily Rolling File Appender
+#
+log4j.appender.DRFA=org.apache.log4j.DailyRollingFileAppender
+log4j.appender.DRFA.File=${hbase.log.dir}/${hbase.log.file}
+
+# Rollver at midnight
+log4j.appender.DRFA.DatePattern=.yyyy-MM-dd
+
+# 30-day backup
+#log4j.appender.DRFA.MaxBackupIndex=30
+log4j.appender.DRFA.layout=org.apache.log4j.PatternLayout
+
+# Pattern format: Date LogLevel LoggerName LogMessage
+log4j.appender.DRFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %m%n
+
+# Rolling File Appender properties
+hbase.log.maxfilesize=256MB
+hbase.log.maxbackupindex=20
+
+# Rolling File Appender
+log4j.appender.RFA=org.apache.log4j.RollingFileAppender
+log4j.appender.RFA.File=${hbase.log.dir}/${hbase.log.file}
+
+log4j.appender.RFA.MaxFileSize=${hbase.log.maxfilesize}
+log4j.appender.RFA.MaxBackupIndex=${hbase.log.maxbackupindex}
+
+log4j.appender.RFA.layout=org.apache.log4j.PatternLayout
+log4j.appender.RFA.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %m%n
+
+#
+# Security audit appender
+#
+hbase.security.log.file=SecurityAuth.audit
+hbase.security.log.maxfilesize=256MB
+hbase.security.log.maxbackupindex=20
+log4j.appender.RFAS=org.apache.log4j.RollingFileAppender
+log4j.appender.RFAS.File=${hbase.log.dir}/${hbase.security.log.file}
+log4j.appender.RFAS.MaxFileSize=${hbase.security.log.maxfilesize}
+log4j.appender.RFAS.MaxBackupIndex=${hbase.security.log.maxbackupindex}
+log4j.appender.RFAS.layout=org.apache.log4j.PatternLayout
+log4j.appender.RFAS.layout.ConversionPattern=%d{ISO8601} %p %c: %m%n
+log4j.category.SecurityLogger=${hbase.security.logger}
+log4j.additivity.SecurityLogger=false
+#log4j.logger.SecurityLogger.org.apache.hadoop.hbase.security.access.AccessController=TRACE
+
+#
+# Null Appender
+#
+log4j.appender.NullAppender=org.apache.log4j.varia.NullAppender
+
+#
+# console
+# Add "console" to rootlogger above if you want to use this
+#
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %m%n
+
+# Custom Logging levels
+
+log4j.logger.org.apache.zookeeper=INFO
+#log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
+log4j.logger.org.apache.hadoop.hbase=DEBUG
+# Make these two classes INFO-level. Make them DEBUG to see more zk debug.
+log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=INFO
+log4j.logger.org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher=INFO
+#log4j.logger.org.apache.hadoop.dfs=DEBUG
+# Set this class to log INFO only otherwise its OTT
+# Enable this to get detailed connection error/retry logging.
+# log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=TRACE
+
+
+# Uncomment this line to enable tracing on _every_ RPC call (this can be a lot of output)
+#log4j.logger.org.apache.hadoop.ipc.HBaseServer.trace=DEBUG
+
+# Uncomment the below if you want to remove logging of client region caching'
+# and scan of .META. messages
+# log4j.logger.org.apache.hadoop.hbase.client.HConnectionManager$HConnectionImplementation=INFO
+# log4j.logger.org.apache.hadoop.hbase.client.MetaScanner=INFO

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/resources/hbase_client_jaas.conf.j2
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/resources/hbase_client_jaas.conf.j2 b/app-packages/hbase-nopkg/resources/hbase_client_jaas.conf.j2
new file mode 100644
index 0000000..bb4279c
--- /dev/null
+++ b/app-packages/hbase-nopkg/resources/hbase_client_jaas.conf.j2
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+Client {
+com.sun.security.auth.module.Krb5LoginModule required
+useKeyTab=false
+useTicketCache=true;
+};

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/resources/hbase_master_jaas.conf.j2
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/resources/hbase_master_jaas.conf.j2 b/app-packages/hbase-nopkg/resources/hbase_master_jaas.conf.j2
new file mode 100644
index 0000000..91ce3ef
--- /dev/null
+++ b/app-packages/hbase-nopkg/resources/hbase_master_jaas.conf.j2
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+Client {
+com.sun.security.auth.module.Krb5LoginModule required
+useKeyTab=true
+storeKey=true
+useTicketCache=false
+keyTab="{{master_keytab_path}}"
+principal="{{master_jaas_princ}}";
+};

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/resources/hbase_regionserver_jaas.conf.j2
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/resources/hbase_regionserver_jaas.conf.j2 b/app-packages/hbase-nopkg/resources/hbase_regionserver_jaas.conf.j2
new file mode 100644
index 0000000..2a9b9f3
--- /dev/null
+++ b/app-packages/hbase-nopkg/resources/hbase_regionserver_jaas.conf.j2
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+Client {
+com.sun.security.auth.module.Krb5LoginModule required
+useKeyTab=true
+storeKey=true
+useTicketCache=false
+keyTab="{{regionserver_keytab_path}}"
+principal="{{regionserver_jaas_princ}}";
+};

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b1cc78a6/app-packages/hbase-nopkg/src/assembly/hbase.xml
----------------------------------------------------------------------
diff --git a/app-packages/hbase-nopkg/src/assembly/hbase.xml b/app-packages/hbase-nopkg/src/assembly/hbase.xml
new file mode 100644
index 0000000..da10fb6
--- /dev/null
+++ b/app-packages/hbase-nopkg/src/assembly/hbase.xml
@@ -0,0 +1,82 @@
+<!--
+  ~ 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.
+  -->
+
+
+<assembly
+  xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.0 http://maven.apache.org/xsd/assembly-1.1.0.xsd">
+  <id>hbase_v${hbase.version}</id>
+  <formats>
+    <format>zip</format>
+    <format>dir</format>
+  </formats>
+  <includeBaseDirectory>false</includeBaseDirectory>
+
+  <files>
+    <file>
+      <source>appConfig-default.json</source>
+      <outputDirectory>/</outputDirectory>
+      <filtered>true</filtered>
+      <fileMode>0755</fileMode>
+    </file>
+    <file>
+      <source>appConfig-secured-default.json</source>
+      <outputDirectory>/</outputDirectory>
+      <filtered>true</filtered>
+      <fileMode>0755</fileMode>
+    </file>
+    <file>
+      <source>metainfo.xml</source>
+      <outputDirectory>/</outputDirectory>
+      <filtered>true</filtered>
+      <fileMode>0755</fileMode>
+    </file>
+    <file>
+      <source>metainfo-secured.xml</source>
+      <outputDirectory>/</outputDirectory>
+      <filtered>true</filtered>
+      <fileMode>0755</fileMode>
+    </file>
+    <file>
+      <source>${pkg.src}/${pkg.name}</source>
+      <outputDirectory>resources</outputDirectory>
+      <filtered>false</filtered>
+      <fileMode>0755</fileMode>
+    </file>
+  </files>
+
+  <fileSets>
+    <fileSet>
+      <directory>${project.basedir}</directory>
+      <outputDirectory>/</outputDirectory>
+      <excludes>
+        <exclude>pom.xml</exclude>
+        <exclude>src/**</exclude>
+        <exclude>target/**</exclude>
+        <exclude>appConfig-default.json</exclude>
+        <exclude>appConfig-secured-default.json</exclude>
+        <exclude>metainfo.xml</exclude>
+        <exclude>metainfo-secured.xml</exclude>
+      </excludes>
+      <fileMode>0755</fileMode>
+      <directoryMode>0755</directoryMode>
+    </fileSet>
+
+  </fileSets>
+</assembly>