You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by sm...@apache.org on 2014/07/11 06:17:41 UTC

[01/50] [abbrv] git commit: SLIDER-171: mark up some methods as clearly not for use x-platform, but only for local setup of miniclusters

Repository: incubator-slider
Updated Branches:
  refs/heads/develop bedf1c1d0 -> 166d25230
  refs/heads/master d595362e0 -> ddf733875


SLIDER-171: mark up some methods as clearly not for use x-platform, but only for local setup of miniclusters


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

Branch: refs/heads/master
Commit: 613dbec829810edff005c9b8aee00eba8f8bc1b8
Parents: 0317c41
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jul 1 21:44:35 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Jul 1 21:44:35 2014 +0100

----------------------------------------------------------------------
 .../org/apache/slider/common/tools/SliderUtils.java |  2 +-
 .../slider/core/launch/ClasspathConstructor.java    | 16 +++++++++++++---
 2 files changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/613dbec8/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
index 62e61c9..7c89321 100644
--- a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
+++ b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
@@ -1144,7 +1144,7 @@ public final class SliderUtils {
     if (usingMiniMRCluster) {
       // for mini cluster we pass down the java CP properties
       // and nothing else
-      classpath.appendAll(classpath.javaVMClasspath());
+      classpath.appendAll(classpath.localJVMClasspath());
     } else {
       classpath.addLibDir(libdir);
       if (sliderConfDir != null) {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/613dbec8/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java b/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
index ca325d1..5b74c03 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
@@ -123,12 +123,12 @@ public class ClasspathConstructor {
 
 
   public void addRemoteClasspathEnvVar() {
-    append(ApplicationConstants.Environment.CLASSPATH.$());
+    append(ApplicationConstants.Environment.CLASSPATH.$$());
   }
 
 
   public void insertRemoteClasspathEnvVar() {
-    append(ApplicationConstants.Environment.CLASSPATH.$());
+    append(ApplicationConstants.Environment.CLASSPATH.$$());
   }
 
 
@@ -152,12 +152,22 @@ public class ClasspathConstructor {
     return dir;
   }
 
+  /**
+   * Split a classpath. This uses the local path separator so MUST NOT
+   * be used to work with remote classpaths
+   * @param localpath local path
+   * @return a splite
+   */
   public Collection<String> splitClasspath(String localpath) {
     String separator = System.getProperty("path.separator");
     return StringUtils.getStringCollection(localpath, separator);
   }
 
-  public Collection<String> javaVMClasspath() {
+  /**
+   * Get the local JVM classpath split up
+   * @return the list of entries on the JVM classpath env var
+   */
+  public Collection<String> localJVMClasspath() {
     return splitClasspath(System.getProperty("java.class.path"));
   }
 


[29/50] [abbrv] git commit: SLIDER-218. Unit Test failure causing build failure (Gour Saha via smohanty)

Posted by sm...@apache.org.
SLIDER-218. Unit Test failure causing build failure (Gour Saha via smohanty)


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

Branch: refs/heads/master
Commit: 3427b6e134bf1cb8a64e40fa840fc816663af48d
Parents: 32c7ea9
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Tue Jul 8 20:30:02 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Tue Jul 8 20:30:02 2014 -0700

----------------------------------------------------------------------
 .../providers/agent/TestBuildBasicAgent.groovy  | 59 +++++++++++++-------
 1 file changed, 40 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3427b6e1/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestBuildBasicAgent.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestBuildBasicAgent.groovy b/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestBuildBasicAgent.groovy
index 9f44888..421920f 100644
--- a/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestBuildBasicAgent.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestBuildBasicAgent.groovy
@@ -235,8 +235,8 @@ class TestBuildBasicAgent extends AgentTestBase {
   }
 
   @Test
-  public void testBadAgentArgs() throws Throwable {
-    def clustername = "test_bad_template_args"
+  public void testAgentArgs() throws Throwable {
+    def clustername = "test_good_agent_args"
     createMiniCluster(
         clustername,
         configuration,
@@ -247,30 +247,43 @@ class TestBuildBasicAgent extends AgentTestBase {
         false)
 
     try {
-      def badArgs1 = "test_bad_agent_args-1"
+      def badArgs1 = "test_good_agent_args-1"
       buildAgentCluster(clustername,
           [:],
           [
               ARG_OPTION, CONTROLLER_URL, "http://localhost",
+              ARG_PACKAGE, ".",
               ARG_OPTION, APP_DEF, "file://" + getAppDef().absolutePath,
-              ARG_OPTION, AGENT_CONF, "file://" + getAgentConf().absolutePath,
               ARG_RESOURCES, TEST_FILES + "good/resources.json",
               ARG_TEMPLATE, TEST_FILES + "good/appconf.json"
           ],
           true, false,
           false)
-      failWithBuildSucceeding(badArgs1, "missing package home or image path")
-    } catch (BadConfigException expected) {
-
+    } catch (BadConfigException exception) {
+      log.error(
+          "Build operation should not have failed with exception : \n$exception")
+      fail("Build operation should not fail")
     }
+  }
+  
+  @Test
+  public void testBadAgentArgs() throws Throwable {
+    def clustername = "test_bad_agent_args"
+    createMiniCluster(
+        clustername,
+        configuration,
+        1,
+        1,
+        1,
+        true,
+        false)
 
     try {
-      def badArgs1 = "test_bad_agent_args-2"
-      buildAgentCluster(clustername,
+      def badArgs1 = "test_bad_agent_args-1"
+      buildAgentCluster(badArgs1,
           [:],
           [
               ARG_OPTION, CONTROLLER_URL, "http://localhost",
-              ARG_IMAGE, "file://" + getAgentImg().absolutePath + ".badfile",
               ARG_OPTION, APP_DEF, "file://" + getAppDef().absolutePath,
               ARG_OPTION, AGENT_CONF, "file://" + getAgentConf().absolutePath,
               ARG_RESOURCES, TEST_FILES + "good/resources.json",
@@ -278,46 +291,54 @@ class TestBuildBasicAgent extends AgentTestBase {
           ],
           true, false,
           false)
-      failWithBuildSucceeding(badArgs1, "bad image path")
+      failWithBuildSucceeding(badArgs1, "missing package home or image path")
     } catch (BadConfigException expected) {
+      log.info("Expected failure.", expected)
+      assert expected.message.contains("Either agent package path agent.package.root or image root internal.application.image.path must be provided")
     }
 
     try {
-      def badArgs1 = "test_bad_agent_args-3"
-      buildAgentCluster(clustername,
+      def badArgs1 = "test_bad_agent_args-2"
+      buildAgentCluster(badArgs1,
           [:],
           [
               ARG_OPTION, CONTROLLER_URL, "http://localhost",
+              ARG_IMAGE, "file://" + getAgentImg().absolutePath,
+              ARG_OPTION, APP_DEF, "file://" + getAppDef().absolutePath,
               ARG_OPTION, AGENT_CONF, "file://" + getAgentConf().absolutePath,
               ARG_RESOURCES, TEST_FILES + "good/resources.json",
               ARG_TEMPLATE, TEST_FILES + "good/appconf.json"
           ],
           true, false,
           false)
-      failWithBuildSucceeding(badArgs1, "bad app def file")
+      failWithBuildSucceeding(badArgs1, "both app image path and home dir was provided")
     } catch (BadConfigException expected) {
+      log.info("Expected failure.", expected)
+      assert expected.message.contains("Both application image path and home dir have been provided")
     }
 
     try {
-      def badArgs1 = "test_bad_agent_args-5"
-      buildAgentCluster(clustername,
+      def badArgs1 = "test_bad_agent_args-3"
+      buildAgentCluster(badArgs1,
           [:],
           [
               ARG_OPTION, CONTROLLER_URL, "http://localhost",
+              ARG_OPTION, AGENT_CONF, "file://" + getAgentConf().absolutePath,
               ARG_PACKAGE, ".",
-              ARG_OPTION, APP_DEF, "file://" + getAppDef().absolutePath,
               ARG_RESOURCES, TEST_FILES + "good/resources.json",
               ARG_TEMPLATE, TEST_FILES + "good/appconf.json"
           ],
           true, false,
           false)
-      failWithBuildSucceeding(badArgs1, "bad agent conf file")
+      failWithBuildSucceeding(badArgs1, "missing app def file")
     } catch (BadConfigException expected) {
+      log.info("Expected failure.", expected)
+      assert expected.message.contains("Application definition must be provided. Missing option application.def")
     }
 
     try {
       def badArgs1 = "test_bad_agent_args-6"
-      buildAgentCluster(clustername,
+      buildAgentCluster(badArgs1,
           [:],
           [
               ARG_OPTION, CONTROLLER_URL, "http://localhost",


[28/50] [abbrv] git commit: SLIDER-181. Storm App Package (minor config changes and folder name changes)

Posted by sm...@apache.org.
SLIDER-181. Storm App Package (minor config changes and folder name changes)


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

Branch: refs/heads/master
Commit: 32c7ea98a8c790852f60fc9d24214f53b1ebd572
Parents: 6f677f6
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Tue Jul 8 17:47:44 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Tue Jul 8 17:47:44 2014 -0700

----------------------------------------------------------------------
 app-packages/storm-v0_91/README.txt             |  33 --
 app-packages/storm-v0_91/appConfig.json         | 125 ----
 .../storm-v0_91/configuration/global.xml        |  39 --
 .../storm-v0_91/configuration/storm-site.xml    | 587 -------------------
 app-packages/storm-v0_91/ganglia_metrics.json   |  31 -
 app-packages/storm-v0_91/jmx_metrics.json       |  31 -
 app-packages/storm-v0_91/metainfo.xml           | 145 -----
 ...pache-storm-0.9.1.2.1.1.0-237.tar.gz.REPLACE |  16 -
 .../storm-v0_91/package/scripts/drpc_server.py  |  55 --
 .../storm-v0_91/package/scripts/nimbus.py       |  55 --
 .../storm-v0_91/package/scripts/params.py       |  58 --
 .../storm-v0_91/package/scripts/rest_api.py     |  57 --
 .../storm-v0_91/package/scripts/service.py      |  95 ---
 .../package/scripts/status_params.py            |  36 --
 .../storm-v0_91/package/scripts/storm.py        |  50 --
 .../storm-v0_91/package/scripts/supervisor.py   |  61 --
 .../storm-v0_91/package/scripts/ui_server.py    |  55 --
 .../storm-v0_91/package/scripts/yaml_config.py  |  69 ---
 .../package/templates/config.yaml.j2            |  48 --
 .../package/templates/storm_jaas.conf.j2        |  26 -
 app-packages/storm-v0_91/resources.json         |  31 -
 app-packages/storm/README.txt                   |  37 ++
 app-packages/storm/appConfig.json               | 126 ++++
 app-packages/storm/configuration/global.xml     |  39 ++
 app-packages/storm/configuration/storm-site.xml | 587 +++++++++++++++++++
 app-packages/storm/ganglia_metrics.json         |  31 +
 app-packages/storm/jmx_metrics.json             |  31 +
 app-packages/storm/metainfo.xml                 | 145 +++++
 ...pache-storm-0.9.1.2.1.1.0-237.tar.gz.REPLACE |  16 +
 .../storm/package/scripts/drpc_server.py        |  55 ++
 app-packages/storm/package/scripts/nimbus.py    |  55 ++
 app-packages/storm/package/scripts/params.py    |  58 ++
 app-packages/storm/package/scripts/rest_api.py  |  57 ++
 app-packages/storm/package/scripts/service.py   |  95 +++
 .../storm/package/scripts/status_params.py      |  36 ++
 app-packages/storm/package/scripts/storm.py     |  50 ++
 .../storm/package/scripts/supervisor.py         |  61 ++
 app-packages/storm/package/scripts/ui_server.py |  55 ++
 .../storm/package/scripts/yaml_config.py        |  69 +++
 .../storm/package/templates/config.yaml.j2      |  48 ++
 .../storm/package/templates/storm_jaas.conf.j2  |  26 +
 app-packages/storm/resources.json               |  31 +
 42 files changed, 1708 insertions(+), 1703 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/README.txt
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/README.txt b/app-packages/storm-v0_91/README.txt
deleted file mode 100644
index b547424..0000000
--- a/app-packages/storm-v0_91/README.txt
+++ /dev/null
@@ -1,33 +0,0 @@
-<!---
-   Licensed to the Apache Software Foundation (ASF) under one or more
-   contributor license agreements.  See the NOTICE file distributed with
-   this work for additional information regarding copyright ownership.
-   The ASF licenses this file to You under the Apache License, Version 2.0
-   (the "License"); you may not use this file except in compliance with
-   the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-   Unless required by applicable law or agreed to in writing, software
-   distributed under the License is distributed on an "AS IS" BASIS,
-   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-   See the License for the specific language governing permissions and
-   limitations under the License.
--->
-
-How to create a Slider package?
-
-Replace the placeholder tarball for Storm.
-  cp ~/Downloads/apache-storm-0.9.1.2.1.1.0-237.tar.gz package/files/
-  rm package/files/apache-storm-0.9.1.2.1.1.0-237.tar.gz.REPLACE
-
-Create a zip package at the root of the package (<slider enlistment>/app-packages/storm-v0_91/) 
-  zip -r storm_v091.zip .
-
-Verify the content using  
-  unzip -l "$@" storm_v091.zip
-
-While appConfig.json and resources.json are not required for the package they work
-well as the default configuration for Slider apps. So its advisable that when you
-create an application package for Slider, include sample/default resources.json and
-appConfig.json for a minimal Yarn cluster.

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/appConfig.json
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/appConfig.json b/app-packages/storm-v0_91/appConfig.json
deleted file mode 100644
index 7ccf412..0000000
--- a/app-packages/storm-v0_91/appConfig.json
+++ /dev/null
@@ -1,125 +0,0 @@
-{
-  "schema": "http://example.org/specification/v2.0.0",
-  "metadata": {
-  },
-  "global": {
-    "application.def": "/slider/storm_v091.zip",
-    "config_types": "storm-site",
-    "java_home": "/usr/jdk64/jdk1.7.0_45",
-    "package_list": "files/apache-storm-0.9.1.2.1.1.0-237.tar.gz",
-    "site.global.app_user": "yarn",
-    "site.global.app_root": "${AGENT_WORK_ROOT}/app/install/apache-storm-0.9.1.2.1.1.0-237",
-    "site.global.user_group": "hadoop",
-    "site.global.security_enabled": "false",
-    "site.global.ganglia_server_host": "${NN_HOST}",
-    "site.global.ganglia_server_id": "Application2",
-    "site.global.ganglia_enabled":"true",
-    "site.global.ganglia_server_port": "8668",
-    "site.global.rest_api_port": "${STORM_REST_API.ALLOCATED_PORT}",
-    "site.global.rest_api_admin_port": "${STORM_REST_API.ALLOCATED_PORT}",
-    "site.storm-site.topology.tuple.serializer": "backtype.storm.serialization.types.ListDelegateSerializer",
-    "site.storm-site.topology.workers": "1",
-    "site.storm-site.drpc.worker.threads": "64",
-    "site.storm-site.storm.zookeeper.servers": "['${ZK_HOST}']",
-    "site.storm-site.supervisor.heartbeat.frequency.secs": "5",
-    "site.storm-site.topology.executor.send.buffer.size": "1024",
-    "site.storm-site.drpc.childopts": "-Xmx768m",
-    "site.storm-site.nimbus.thrift.port": "${NIMBUS.ALLOCATED_PORT}",
-    "site.storm-site.storm.zookeeper.retry.intervalceiling.millis": "30000",
-    "site.storm-site.storm.local.dir": "${AGENT_WORK_ROOT}/app/tmp/storm",
-    "site.storm-site.topology.receiver.buffer.size": "8",
-    "site.storm-site.storm.messaging.netty.client_worker_threads": "1",
-    "site.storm-site.transactional.zookeeper.root": "/transactional",
-    "site.storm-site.drpc.request.timeout.secs": "600",
-    "site.storm-site.topology.skip.missing.kryo.registrations": "false",
-    "site.storm-site.worker.heartbeat.frequency.secs": "1",
-    "site.storm-site.zmq.hwm": "0",
-    "site.storm-site.storm.zookeeper.connection.timeout": "15000",
-    "site.storm-site.topology.max.error.report.per.interval": "5",
-    "site.storm-site.storm.messaging.netty.server_worker_threads": "1",
-    "site.storm-site.supervisor.worker.start.timeout.secs": "120",
-    "site.storm-site.zmq.threads": "1",
-    "site.storm-site.topology.acker.executors": "null",
-    "site.storm-site.storm.local.mode.zmq": "false",
-    "site.storm-site.topology.max.task.parallelism": "null",
-    "site.storm-site.storm.zookeeper.port": "2181",
-    "site.storm-site.nimbus.childopts": "-Xmx1024m -javaagent:${AGENT_WORK_ROOT}/app/install/apache-storm-0.9.1.2.1.1.0-237/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=${NN_HOST},port=8668,wireformat31x=true,mode=multicast,config=${AGENT_WORK_ROOT}/app/install/apache-storm-0.9.1.2.1.1.0-237/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Nimbus_JVM",
-    "site.storm-site.worker.childopts": "-Xmx768m -javaagent:${AGENT_WORK_ROOT}/app/install/apache-storm-0.9.1.2.1.1.0-237/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=${NN_HOST},port=8668,wireformat31x=true,mode=multicast,config=${AGENT_WORK_ROOT}/app/install/apache-storm-0.9.1.2.1.1.0-237/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Worker_%ID%_JVM",
-    "site.storm-site.drpc.queue.size": "128",
-    "site.storm-site.storm.zookeeper.retry.times": "5",
-    "site.storm-site.nimbus.monitor.freq.secs": "10",
-    "site.storm-site.storm.cluster.mode": "distributed",
-    "site.storm-site.dev.zookeeper.path": "${AGENT_WORK_ROOT}/app/tmp/dev-storm-zookeeper",
-    "site.storm-site.drpc.invocations.port": "0",
-    "site.storm-site.storm.zookeeper.root": "/storm",
-    "site.storm-site.logviewer.childopts": "-Xmx128m",
-    "site.storm-site.transactional.zookeeper.port": "null",
-    "site.storm-site.topology.worker.childopts": "null",
-    "site.storm-site.topology.max.spout.pending": "null",
-    "site.storm-site.nimbus.cleanup.inbox.freq.secs": "600",
-    "site.storm-site.storm.messaging.netty.min_wait_ms": "100",
-    "site.storm-site.nimbus.task.timeout.secs": "30",
-    "site.storm-site.nimbus.thrift.max_buffer_size": "1048576",
-    "site.storm-site.topology.sleep.spout.wait.strategy.time.ms": "1",
-    "site.storm-site.topology.optimize": "true",
-    "site.storm-site.nimbus.reassign": "true",
-    "site.storm-site.storm.messaging.transport": "backtype.storm.messaging.netty.Context",
-    "site.storm-site.logviewer.appender.name": "A1",
-    "site.storm-site.nimbus.host": "${NIMBUS_HOST}",
-    "site.storm-site.ui.port": "${STORM_UI_SERVER.ALLOCATED_PORT}",
-    "site.storm-site.supervisor.slots.ports": "[0, 0]",
-    "site.storm-site.nimbus.file.copy.expiration.secs": "600",
-    "site.storm-site.supervisor.monitor.frequency.secs": "3",
-    "site.storm-site.transactional.zookeeper.servers": "null",
-    "site.storm-site.zmq.linger.millis": "5000",
-    "site.storm-site.topology.error.throttle.interval.secs": "10",
-    "site.storm-site.topology.worker.shared.thread.pool.size": "4",
-    "site.storm-site.java.library.path": "/usr/local/lib:/opt/local/lib:/usr/lib",
-    "site.storm-site.topology.spout.wait.strategy": "backtype.storm.spout.SleepSpoutWaitStrategy",
-    "site.storm-site.task.heartbeat.frequency.secs": "3",
-    "site.storm-site.topology.transfer.buffer.size": "1024",
-    "site.storm-site.storm.zookeeper.session.timeout": "20000",
-    "site.storm-site.topology.executor.receive.buffer.size": "1024",
-    "site.storm-site.topology.stats.sample.rate": "0.05",
-    "site.storm-site.topology.fall.back.on.java.serialization": "true",
-    "site.storm-site.supervisor.childopts": "-Xmx256m -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.port=0 -javaagent:${AGENT_WORK_ROOT}/app/install/apache-storm-0.9.1.2.1.1.0-237/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=${NN_HOST},port=8668,wireformat31x=true,mode=multicast,config=${AGENT_WORK_ROOT}/app/install/apache-storm-0.9.1.2.1.1.0-237/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Supervisor_JVM",
-    "site.storm-site.topology.enable.message.timeouts": "true",
-    "site.storm-site.storm.messaging.netty.max_wait_ms": "1000",
-    "site.storm-site.nimbus.topology.validator": "backtype.storm.nimbus.DefaultTopologyValidator",
-    "site.storm-site.nimbus.supervisor.timeout.secs": "60",
-    "site.storm-site.topology.disruptor.wait.strategy": "com.lmax.disruptor.BlockingWaitStrategy",
-    "site.storm-site.nimbus.inbox.jar.expiration.secs": "3600",
-    "site.storm-site.drpc.port": "0",
-    "site.storm-site.topology.kryo.factory": "backtype.storm.serialization.DefaultKryoFactory",
-    "site.storm-site.storm.zookeeper.retry.interval": "1000",
-    "site.storm-site.storm.messaging.netty.max_retries": "30",
-    "site.storm-site.topology.tick.tuple.freq.secs": "null",
-    "site.storm-site.supervisor.enable": "true",
-    "site.storm-site.nimbus.task.launch.secs": "120",
-    "site.storm-site.task.refresh.poll.secs": "10",
-    "site.storm-site.topology.message.timeout.secs": "30",
-    "site.storm-site.storm.messaging.netty.buffer_size": "5242880",
-    "site.storm-site.topology.state.synchronization.timeout.secs": "60",
-    "site.storm-site.supervisor.worker.timeout.secs": "30",
-    "site.storm-site.topology.trident.batch.emit.interval.millis": "500",
-    "site.storm-site.topology.builtin.metrics.bucket.size.secs": "60",
-    "site.storm-site.storm.thrift.transport": "backtype.storm.security.auth.SimpleTransportPlugin",
-    "site.storm-site.logviewer.port": "0",
-    "site.storm-site.topology.debug": "false"
-  },
-  "components": {
-    "slider-appmaster": {
-      "jvm.heapsize": "256M"
-    },
-    "NIMBUS": {
-    },
-    "STORM_REST_API": {
-    },
-    "STORM_UI_SERVER": {
-    },
-    "DRPC_SERVER": {
-    },
-    "SUPERVISOR": {
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/configuration/global.xml
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/configuration/global.xml b/app-packages/storm-v0_91/configuration/global.xml
deleted file mode 100644
index 5cc9170..0000000
--- a/app-packages/storm-v0_91/configuration/global.xml
+++ /dev/null
@@ -1,39 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-/**
- * 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.
- */
--->
-
-<configuration>
-  <property>
-    <name>storm_user</name>
-    <value>storm</value>
-    <description></description>
-  </property>
-  <property>
-    <name>storm_log_dir</name>
-    <value>/var/log/storm</value>
-    <description></description>
-  </property>
-  <property>
-    <name>storm_pid_dir</name>
-    <value>/var/run/storm</value>
-    <description></description>
-  </property>
-</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/configuration/storm-site.xml
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/configuration/storm-site.xml b/app-packages/storm-v0_91/configuration/storm-site.xml
deleted file mode 100644
index 6eca8f9..0000000
--- a/app-packages/storm-v0_91/configuration/storm-site.xml
+++ /dev/null
@@ -1,587 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-/**
- * 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.
- */
--->
-
-<configuration>
-  <property>
-    <name>java.library.path</name>
-    <value>/usr/local/lib:/opt/local/lib:/usr/lib</value>
-    <description>This value is passed to spawned JVMs (e.g., Nimbus, Supervisor, and Workers)
-       for the java.library.path value. java.library.path tells the JVM where
-       to look for native libraries. It is necessary to set this config correctly since
-       Storm uses the ZeroMQ and JZMQ native libs. </description>
-  </property>
-  <property>
-    <name>storm.local.dir</name>
-    <value>/hadoop/storm</value>
-    <description>A directory on the local filesystem used by Storm for any local
-       filesystem usage it needs. The directory must exist and the Storm daemons must
-       have permission to read/write from this location.</description>
-  </property>
-  <property>
-    <name>storm.zookeeper.servers</name>
-    <value>['localhost']</value>
-    <description>A list of hosts of ZooKeeper servers used to manage the cluster.</description>
-  </property>
-  <property>
-    <name>storm.zookeeper.port</name>
-    <value>2181</value>
-    <description>The port Storm will use to connect to each of the ZooKeeper servers.</description>
-  </property>
-  <property>
-    <name>storm.zookeeper.root</name>
-    <value>/storm</value>
-    <description>The root location at which Storm stores data in ZooKeeper.</description>
-  </property>
-  <property>
-    <name>storm.zookeeper.session.timeout</name>
-    <value>20000</value>
-    <description>The session timeout for clients to ZooKeeper.</description>
-  </property>
-  <property>
-    <name>storm.zookeeper.connection.timeout</name>
-    <value>15000</value>
-    <description>The connection timeout for clients to ZooKeeper.</description>
-  </property>
-  <property>
-    <name>storm.zookeeper.retry.times</name>
-    <value>5</value>
-    <description>The number of times to retry a Zookeeper operation.</description>
-  </property>
-  <property>
-    <name>storm.zookeeper.retry.interval</name>
-    <value>1000</value>
-    <description>The interval between retries of a Zookeeper operation.</description>
-  </property>
-  <property>
-    <name>storm.zookeeper.retry.intervalceiling.millis</name>
-    <value>30000</value>
-    <description>The ceiling of the interval between retries of a Zookeeper operation.</description>
-  </property>
-  <property>
-    <name>storm.cluster.mode</name>
-    <value>distributed</value>
-    <description>The mode this Storm cluster is running in. Either "distributed" or "local".</description>
-  </property>
-  <property>
-    <name>storm.local.mode.zmq</name>
-    <value>false</value>
-    <description>Whether or not to use ZeroMQ for messaging in local mode. If this is set
-       to false, then Storm will use a pure-Java messaging system. The purpose
-       of this flag is to make it easy to run Storm in local mode by eliminating
-       the need for native dependencies, which can be difficult to install.
-    </description>
-  </property>
-  <property>
-    <name>storm.thrift.transport</name>
-    <value>backtype.storm.security.auth.SimpleTransportPlugin</value>
-    <description>The transport plug-in for Thrift client/server communication.</description>
-  </property>
-  <property>
-    <name>storm.messaging.transport</name>
-    <value>backtype.storm.messaging.netty.Context</value>
-    <description>The transporter for communication among Storm tasks.</description>
-  </property>
-  <property>
-    <name>nimbus.host</name>
-    <value>localhost</value>
-    <description>The host that the master server is running on.</description>
-  </property>
-  <property>
-    <name>nimbus.thrift.port</name>
-    <value>6627</value>
-    <description> Which port the Thrift interface of Nimbus should run on. Clients should
-       connect to this port to upload jars and submit topologies.</description>
-  </property>
-  <property>
-    <name>nimbus.thrift.max_buffer_size</name>
-    <value>1048576</value>
-    <description>The maximum buffer size thrift should use when reading messages.</description>
-  </property>
-  <property>
-    <name>nimbus.childopts</name>
-    <value>-Xmx1024m -Djava.security.auth.login.config=/etc/storm/storm_jaas.conf -javaagent:/usr/lib/storm/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host={0},port=8649,wireformat31x=true,mode=multicast,config=/usr/lib/storm/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Nimbus_JVM</value>
-    <description>This parameter is used by the storm-deploy project to configure the jvm options for the nimbus daemon.</description>
-  </property>
-  <property>
-    <name>nimbus.task.timeout.secs</name>
-    <value>30</value>
-    <description>How long without heartbeating a task can go before nimbus will consider the task dead and reassign it to another location.</description>
-  </property>
-  <property>
-    <name>nimbus.supervisor.timeout.secs</name>
-    <value>60</value>
-    <description>How long before a supervisor can go without heartbeating before nimbus considers it dead and stops assigning new work to it.</description>
-  </property>
-  <property>
-    <name>nimbus.monitor.freq.secs</name>
-    <value>10</value>
-    <description>
-      How often nimbus should wake up to check heartbeats and do reassignments. Note
-       that if a machine ever goes down Nimbus will immediately wake up and take action.
-       This parameter is for checking for failures when there's no explicit event like that occuring.
-    </description>
-  </property>
-  <property>
-    <name>nimbus.cleanup.inbox.freq.secs</name>
-    <value>600</value>
-    <description>How often nimbus should wake the cleanup thread to clean the inbox.</description>
-  </property>
-  <property>
-    <name>nimbus.inbox.jar.expiration.secs</name>
-    <value>3600</value>
-    <description>
-      The length of time a jar file lives in the inbox before being deleted by the cleanup thread.
-
-       Probably keep this value greater than or equal to NIMBUS_CLEANUP_INBOX_JAR_EXPIRATION_SECS.
-       Note that the time it takes to delete an inbox jar file is going to be somewhat more than
-       NIMBUS_CLEANUP_INBOX_JAR_EXPIRATION_SECS (depending on how often NIMBUS_CLEANUP_FREQ_SECS is set to).
-      </description>
-  </property>
-  <property>
-    <name>nimbus.task.launch.secs</name>
-    <value>120</value>
-    <description>A special timeout used when a task is initially launched. During launch, this is the timeout
-       used until the first heartbeat, overriding nimbus.task.timeout.secs.</description>
-  </property>
-  <property>
-    <name>nimbus.reassign</name>
-    <value>true</value>
-    <description>Whether or not nimbus should reassign tasks if it detects that a task goes down.
-       Defaults to true, and it's not recommended to change this value.</description>
-  </property>
-  <property>
-    <name>nimbus.file.copy.expiration.secs</name>
-    <value>600</value>
-    <description>During upload/download with the master, how long an upload or download connection is idle
-       before nimbus considers it dead and drops the connection.</description>
-  </property>
-  <property>
-    <name>nimbus.topology.validator</name>
-    <value>backtype.storm.nimbus.DefaultTopologyValidator</value>
-    <description>A custom class that implements ITopologyValidator that is run whenever a
-       topology is submitted. Can be used to provide business-specific logic for
-       whether topologies are allowed to run or not.</description>
-  </property>
-  <property>
-    <name>ui.port</name>
-    <value>8744</value>
-    <description>Storm UI binds to this port.</description>
-  </property>
-  <property>
-    <name>ui.childopts</name>
-    <value>-Xmx768m -Djava.security.auth.login.config=/etc/storm/storm_jaas.conf</value>
-    <description>Childopts for Storm UI Java process.</description>
-  </property>
-  <property>
-    <name>logviewer.port</name>
-    <value>8000</value>
-    <description>HTTP UI port for log viewer.</description>
-  </property>
-  <property>
-    <name>logviewer.childopts</name>
-    <value>-Xmx128m</value>
-    <description>Childopts for log viewer java process.</description>
-  </property>
-  <property>
-    <name>logviewer.appender.name</name>
-    <value>A1</value>
-    <description>Appender name used by log viewer to determine log directory.</description>
-  </property>
-  <property>
-    <name>drpc.port</name>
-    <value>3772</value>
-    <description>This port is used by Storm DRPC for receiving DPRC requests from clients.</description>
-  </property>
-  <property>
-    <name>drpc.worker.threads</name>
-    <value>64</value>
-    <description>DRPC thrift server worker threads.</description>
-  </property>
-  <property>
-    <name>drpc.queue.size</name>
-    <value>128</value>
-    <description>DRPC thrift server queue size.</description>
-  </property>
-  <property>
-    <name>drpc.invocations.port</name>
-    <value>3773</value>
-    <description>This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back.</description>
-  </property>
-  <property>
-    <name>drpc.request.timeout.secs</name>
-    <value>600</value>
-    <description>The timeout on DRPC requests within the DRPC server. Defaults to 10 minutes. Note that requests can also
-       timeout based on the socket timeout on the DRPC client, and separately based on the topology message
-       timeout for the topology implementing the DRPC function.</description>
-  </property>
-  <property>
-    <name>drpc.childopts</name>
-    <value>-Xmx768m</value>
-    <description>Childopts for Storm DRPC Java process.</description>
-  </property>
-  <property>
-    <name>transactional.zookeeper.root</name>
-    <value>/transactional</value>
-    <description>The root directory in ZooKeeper for metadata about TransactionalSpouts.</description>
-  </property>
-  <property>
-    <name>transactional.zookeeper.servers</name>
-    <value>null</value>
-    <description>The list of zookeeper servers in which to keep the transactional state. If null (which is default),
-       will use storm.zookeeper.servers</description>
-  </property>
-  <property>
-    <name>transactional.zookeeper.port</name>
-    <value>null</value>
-    <description>The port to use to connect to the transactional zookeeper servers. If null (which is default),
-       will use storm.zookeeper.port</description>
-  </property>
-  <property>
-    <name>supervisor.slots.ports</name>
-    <value>[6700, 6701]</value>
-    <description>A list of ports that can run workers on this supervisor. Each worker uses one port, and
-       the supervisor will only run one worker per port. Use this configuration to tune
-       how many workers run on each machine.</description>
-  </property>
-  <property>
-    <name>supervisor.childopts</name>
-    <value>-Xmx256m -Djava.security.auth.login.config=/etc/storm/storm_jaas.conf -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.port=56431 -javaagent:/usr/lib/storm/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host={0},port=8650,wireformat31x=true,mode=multicast,config=/usr/lib/storm/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Supervisor_JVM</value>
-    <description>This parameter is used by the storm-deploy project to configure the jvm options for the supervisor daemon.</description>
-  </property>
-  <property>
-    <name>supervisor.worker.start.timeout.secs</name>
-    <value>120</value>
-    <description>How long a worker can go without heartbeating during the initial launch before
-       the supervisor tries to restart the worker process. This value override
-       supervisor.worker.timeout.secs during launch because there is additional
-       overhead to starting and configuring the JVM on launch.</description>
-  </property>
-  <property>
-    <name>supervisor.worker.timeout.secs</name>
-    <value>30</value>
-    <description>How long a worker can go without heartbeating before the supervisor tries to restart the worker process.</description>
-  </property>
-  <property>
-    <name>supervisor.monitor.frequency.secs</name>
-    <value>3</value>
-    <description>How often the supervisor checks the worker heartbeats to see if any of them need to be restarted.</description>
-  </property>
-  <property>
-    <name>supervisor.heartbeat.frequency.secs</name>
-    <value>5</value>
-    <description>How often the supervisor sends a heartbeat to the master.</description>
-  </property>
-  <property>
-    <name>supervisor.enable</name>
-    <value>true</value>
-    <description>Whether or not the supervisor should launch workers assigned to it. Defaults
-       to true -- and you should probably never change this value. This configuration
-       is used in the Storm unit tests.</description>
-  </property>
-  <property>
-    <name>worker.childopts</name>
-    <value>-Xmx768m -javaagent:/usr/lib/storm/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host={0},port=8650,wireformat31x=true,mode=multicast,config=/usr/lib/storm/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Worker_%ID%_JVM</value>
-    <description>The jvm opts provided to workers launched by this supervisor. All \"%ID%\" substrings are replaced with an identifier for this worker.</description>
-  </property>
-  <property>
-    <name>worker.heartbeat.frequency.secs</name>
-    <value>1</value>
-    <description>How often this worker should heartbeat to the supervisor.</description>
-  </property>
-  <property>
-    <name>task.heartbeat.frequency.secs</name>
-    <value>3</value>
-    <description>How often a task should heartbeat its status to the master.</description>
-  </property>
-  <property>
-    <name>task.refresh.poll.secs</name>
-    <value>10</value>
-    <description>How often a task should sync its connections with other tasks (if a task is
-       reassigned, the other tasks sending messages to it need to refresh their connections).
-       In general though, when a reassignment happens other tasks will be notified
-       almost immediately. This configuration is here just in case that notification doesn't
-       come through.</description>
-  </property>
-  <property>
-    <name>zmq.threads</name>
-    <value>1</value>
-    <description>The number of threads that should be used by the zeromq context in each worker process.</description>
-  </property>
-  <property>
-    <name>zmq.linger.millis</name>
-    <value>5000</value>
-    <description>How long a connection should retry sending messages to a target host when
-       the connection is closed. This is an advanced configuration and can almost
-       certainly be ignored.</description>
-  </property>
-  <property>
-    <name>zmq.hwm</name>
-    <value>0</value>
-    <description>The high water for the ZeroMQ push sockets used for networking. Use this config to prevent buffer explosion
-       on the networking layer.</description>
-  </property>
-  <property>
-    <name>storm.messaging.netty.server_worker_threads</name>
-    <value>1</value>
-    <description>Netty based messaging: The # of worker threads for the server.</description>
-  </property>
-  <property>
-    <name>storm.messaging.netty.client_worker_threads</name>
-    <value>1</value>
-    <description>Netty based messaging: The # of worker threads for the client.</description>
-  </property>
-  <property>
-    <name>storm.messaging.netty.buffer_size</name>
-    <value>5242880</value>
-    <description>Netty based messaging: The buffer size for send/recv buffer.</description>
-  </property>
-  <property>
-    <name>storm.messaging.netty.max_retries</name>
-    <value>30</value>
-    <description>Netty based messaging: The max # of retries that a peer will perform when a remote is not accessible.</description>
-  </property>
-  <property>
-    <name>storm.messaging.netty.max_wait_ms</name>
-    <value>1000</value>
-    <description>Netty based messaging: The max # of milliseconds that a peer will wait.</description>
-  </property>
-  <property>
-    <name>storm.messaging.netty.min_wait_ms</name>
-    <value>100</value>
-    <description>Netty based messaging: The min # of milliseconds that a peer will wait.</description>
-  </property>
-  <property>
-    <name>topology.enable.message.timeouts</name>
-    <value>true</value>
-    <description>True if Storm should timeout messages or not. Defaults to true. This is meant to be used
-       in unit tests to prevent tuples from being accidentally timed out during the test.</description>
-  </property>
-  <property>
-    <name>topology.debug</name>
-    <value>false</value>
-    <description>When set to true, Storm will log every message that's emitted.</description>
-  </property>
-  <property>
-    <name>topology.optimize</name>
-    <value>true</value>
-    <description>Whether or not the master should optimize topologies by running multiple tasks in a single thread where appropriate.</description>
-  </property>
-  <property>
-    <name>topology.workers</name>
-    <value>1</value>
-    <description>How many processes should be spawned around the cluster to execute this
-       topology. Each process will execute some number of tasks as threads within
-       them. This parameter should be used in conjunction with the parallelism hints
-       on each component in the topology to tune the performance of a topology.</description>
-  </property>
-  <property>
-    <name>topology.acker.executors</name>
-    <value>null</value>
-    <description>How many executors to spawn for ackers.
-
-      If this is set to 0, then Storm will immediately ack tuples as soon
-       as they come off the spout, effectively disabling reliability.
-    </description>
-  </property>
-  <property>
-    <name>topology.message.timeout.secs</name>
-    <value>30</value>
-    <description>The maximum amount of time given to the topology to fully process a message
-       emitted by a spout. If the message is not acked within this time frame, Storm
-       will fail the message on the spout. Some spouts implementations will then replay
-       the message at a later time.</description>
-  </property>
-  <property>
-    <name>topology.skip.missing.kryo.registrations</name>
-    <value>false</value>
-    <description> Whether or not Storm should skip the loading of kryo registrations for which it
-       does not know the class or have the serializer implementation. Otherwise, the task will
-       fail to load and will throw an error at runtime. The use case of this is if you want to
-       declare your serializations on the storm.yaml files on the cluster rather than every single
-       time you submit a topology. Different applications may use different serializations and so
-       a single application may not have the code for the other serializers used by other apps.
-       By setting this config to true, Storm will ignore that it doesn't have those other serializations
-       rather than throw an error.</description>
-  </property>
-  <property>
-    <name>topology.max.task.parallelism</name>
-    <value>null</value>
-    <description>The maximum parallelism allowed for a component in this topology. This configuration is
-       typically used in testing to limit the number of threads spawned in local mode.</description>
-  </property>
-  <property>
-    <name>topology.max.spout.pending</name>
-    <value>null</value>
-    <description>The maximum number of tuples that can be pending on a spout task at any given time.
-       This config applies to individual tasks, not to spouts or topologies as a whole.
-
-       A pending tuple is one that has been emitted from a spout but has not been acked or failed yet.
-       Note that this config parameter has no effect for unreliable spouts that don't tag
-       their tuples with a message id.</description>
-  </property>
-  <property>
-    <name>topology.state.synchronization.timeout.secs</name>
-    <value>60</value>
-    <description>The maximum amount of time a component gives a source of state to synchronize before it requests
-       synchronization again.</description>
-  </property>
-  <property>
-    <name>topology.stats.sample.rate</name>
-    <value>0.05</value>
-    <description>The percentage of tuples to sample to produce stats for a task.</description>
-  </property>
-  <property>
-    <name>topology.builtin.metrics.bucket.size.secs</name>
-    <value>60</value>
-    <description>The time period that builtin metrics data in bucketed into.</description>
-  </property>
-  <property>
-    <name>topology.fall.back.on.java.serialization</name>
-    <value>true</value>
-    <description>Whether or not to use Java serialization in a topology.</description>
-  </property>
-  <property>
-    <name>topology.worker.childopts</name>
-    <value>null</value>
-    <description>Topology-specific options for the worker child process. This is used in addition to WORKER_CHILDOPTS.</description>
-  </property>
-  <property>
-    <name>topology.executor.receive.buffer.size</name>
-    <value>1024</value>
-    <description>The size of the Disruptor receive queue for each executor. Must be a power of 2.</description>
-  </property>
-  <property>
-    <name>topology.executor.send.buffer.size</name>
-    <value>1024</value>
-    <description>The size of the Disruptor send queue for each executor. Must be a power of 2.</description>
-  </property>
-  <property>
-    <name>topology.receiver.buffer.size</name>
-    <value>8</value>
-    <description>The maximum number of messages to batch from the thread receiving off the network to the
-       executor queues. Must be a power of 2.</description>
-  </property>
-  <property>
-    <name>topology.transfer.buffer.size</name>
-    <value>1024</value>
-    <description>The size of the Disruptor transfer queue for each worker.</description>
-  </property>
-  <property>
-    <name>topology.tick.tuple.freq.secs</name>
-    <value>null</value>
-    <description>How often a tick tuple from the "__system" component and "__tick" stream should be sent
-       to tasks. Meant to be used as a component-specific configuration.</description>
-  </property>
-  <property>
-    <name>topology.worker.shared.thread.pool.size</name>
-    <value>4</value>
-    <description>The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed
-       via the TopologyContext.</description>
-  </property>
-  <property>
-    <name>topology.disruptor.wait.strategy</name>
-    <value>com.lmax.disruptor.BlockingWaitStrategy</value>
-    <description>Configure the wait strategy used for internal queuing. Can be used to tradeoff latency
-       vs. throughput.</description>
-  </property>
-  <property>
-    <name>topology.executor.send.buffer.size</name>
-    <value>1024</value>
-    <description>The size of the Disruptor send queue for each executor. Must be a power of 2.</description>
-  </property>
-  <property>
-    <name>topology.receiver.buffer.size</name>
-    <value>8</value>
-    <description>The maximum number of messages to batch from the thread receiving off the network to the
-       executor queues. Must be a power of 2.</description>
-  </property>
-  <property>
-    <name>topology.transfer.buffer.size</name>
-    <value>1024</value>
-    <description>The size of the Disruptor transfer queue for each worker.</description>
-  </property>
-  <property>
-    <name>topology.tick.tuple.freq.secs</name>
-    <value>null</value>
-    <description>How often a tick tuple from the "__system" component and "__tick" stream should be sent
-       to tasks. Meant to be used as a component-specific configuration.</description>
-  </property>
-  <property>
-    <name>topology.worker.shared.thread.pool.size</name>
-    <value>4</value>
-    <description>The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed
-       via the TopologyContext.</description>
-  </property>
-  <property>
-    <name>topology.spout.wait.strategy</name>
-    <value>backtype.storm.spout.SleepSpoutWaitStrategy</value>
-    <description>A class that implements a strategy for what to do when a spout needs to wait. Waiting is
-       triggered in one of two conditions:
-
-       1. nextTuple emits no tuples
-       2. The spout has hit maxSpoutPending and can't emit any more tuples</description>
-  </property>
-  <property>
-    <name>topology.sleep.spout.wait.strategy.time.ms</name>
-    <value>1</value>
-    <description>The amount of milliseconds the SleepEmptyEmitStrategy should sleep for.</description>
-  </property>
-  <property>
-    <name>topology.error.throttle.interval.secs</name>
-    <value>10</value>
-    <description>The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example,
-       an interval of 10 seconds with topology.max.error.report.per.interval set to 5 will only allow 5 errors to be
-       reported to Zookeeper per task for every 10 second interval of time.</description>
-  </property>
-  <property>
-    <name>topology.max.error.report.per.interval</name>
-    <value>5</value>
-    <description>The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example,
-       an interval of 10 seconds with topology.max.error.report.per.interval set to 5 will only allow 5 errors to be
-       reported to Zookeeper per task for every 10 second interval of time.</description>
-  </property>
-  <property>
-    <name>topology.kryo.factory</name>
-    <value>backtype.storm.serialization.DefaultKryoFactory</value>
-    <description>Class that specifies how to create a Kryo instance for serialization. Storm will then apply
-       topology.kryo.register and topology.kryo.decorators on top of this. The default implementation
-       implements topology.fall.back.on.java.serialization and turns references off.</description>
-  </property>
-  <property>
-    <name>topology.tuple.serializer</name>
-    <value>backtype.storm.serialization.types.ListDelegateSerializer</value>
-    <description>The serializer class for ListDelegate (tuple payload).
-       The default serializer will be ListDelegateSerializer</description>
-  </property>
-  <property>
-    <name>topology.trident.batch.emit.interval.millis</name>
-    <value>500</value>
-    <description>How often a batch can be emitted in a Trident topology.</description>
-  </property>
-  <property>
-    <name>dev.zookeeper.path</name>
-    <value>/tmp/dev-storm-zookeeper</value>
-    <description>The path to use as the zookeeper dir when running a zookeeper server via
-       "storm dev-zookeeper". This zookeeper instance is only intended for development;
-       it is not a production grade zookeeper setup.</description>
-  </property>
-</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/ganglia_metrics.json
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/ganglia_metrics.json b/app-packages/storm-v0_91/ganglia_metrics.json
deleted file mode 100644
index 861c4fa..0000000
--- a/app-packages/storm-v0_91/ganglia_metrics.json
+++ /dev/null
@@ -1,31 +0,0 @@
-{
-    "Component": {
-        "NIMBUS": {
-            "totalslots": {
-                "metric": "Total Slots",
-                "pointInTime": false,
-                "temporal": true
-            },
-            "totalexecutors": {
-                "metric": "Total Executors",
-                "pointInTime": false,
-                "temporal": true
-            },
-            "topologies": {
-                "metric": "Topologies",
-                "pointInTime": false,
-                "temporal": true
-            },
-            "totaltasks": {
-                "metric": "Total Tasks",
-                "pointInTime": false,
-                "temporal": true
-            },
-            "usedslots": {
-                "metric": "Used Slots",
-                "pointInTime": false,
-                "temporal": true
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/jmx_metrics.json
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/jmx_metrics.json b/app-packages/storm-v0_91/jmx_metrics.json
deleted file mode 100644
index f7d4e60..0000000
--- a/app-packages/storm-v0_91/jmx_metrics.json
+++ /dev/null
@@ -1,31 +0,0 @@
-{
-    "Component": {
-        "NIMBUS": {
-            "FreeSlots": {
-                "metric": "$['slots.free']",
-                "pointInTime": true,
-                "temporal": false
-            },
-            "Tasks": {
-                "metric": "$['tasks.total']",
-                "pointInTime": true,
-                "temporal": false
-            },
-            "Executors": {
-                "metric": "$['executors.total']",
-                "pointInTime": true,
-                "temporal": false
-            },
-            "Topologies": {
-                "metric": "$['topologies']",
-                "pointInTime": true,
-                "temporal": false
-            },
-            "NimbusUptime": {
-                "metric": "$['nimbus.uptime']",
-                "pointInTime": true,
-                "temporal": false
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/metainfo.xml
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/metainfo.xml b/app-packages/storm-v0_91/metainfo.xml
deleted file mode 100644
index 7edd794..0000000
--- a/app-packages/storm-v0_91/metainfo.xml
+++ /dev/null
@@ -1,145 +0,0 @@
-<?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>STORM</name>
-    <comment>Apache Hadoop Stream processing framework</comment>
-    <version>0.9.1.2.1</version>
-
-    <exportGroups>
-      <exportGroup>
-        <name>QuickLinks</name>
-        <exports>
-          <export>
-            <name>org.apache.slider.jmx</name>
-            <value>http://${STORM_REST_API_HOST}:${site.global.rest_api_port}/api/cluster/summary</value>
-          </export>
-          <export>
-            <name>org.apache.slider.monitor</name>
-            <value>http://${STORM_UI_SERVER_HOST}:${site.storm-site.ui.port}</value>
-          </export>
-          <export>
-            <name>org.apache.slider.metrics</name>
-            <value>http://${site.global.ganglia_server_host}/cgi-bin/rrd.py?c=${site.global.ganglia_server_id}</value>
-          </export>
-          <export>
-            <name>org.apache.slider.ganglia</name>
-            <value>http://${site.global.ganglia_server_host}/ganglia?c=${site.global.ganglia_server_id}</value>
-          </export>
-        </exports>
-      </exportGroup>
-    </exportGroups>
-
-    <commandOrders>
-      <commandOrder>
-        <command>NIMBUS-START</command>
-        <requires>SUPERVISOR-INSTALLED,STORM_UI_SERVER-INSTALLED,DRPC_SERVER-INSTALLED,STORM_REST_API-INSTALLED
-        </requires>
-      </commandOrder>
-      <commandOrder>
-        <command>SUPERVISOR-START</command>
-        <requires>NIMBUS-STARTED</requires>
-      </commandOrder>
-      <commandOrder>
-        <command>DRPC_SERVER-START</command>
-        <requires>NIMBUS-STARTED</requires>
-      </commandOrder>
-      <commandOrder>
-        <command>STORM_REST_API-START</command>
-        <requires>NIMBUS-STARTED,DRPC_SERVER-STARTED,STORM_UI_SERVER-STARTED</requires>
-      </commandOrder>
-      <commandOrder>
-        <command>STORM_UI_SERVER-START</command>
-        <requires>NIMBUS-STARTED</requires>
-      </commandOrder>
-    </commandOrders>
-
-    <components>
-
-      <component>
-        <name>NIMBUS</name>
-        <category>MASTER</category>
-        <commandScript>
-          <script>scripts/nimbus.py</script>
-          <scriptType>PYTHON</scriptType>
-          <timeout>600</timeout>
-        </commandScript>
-      </component>
-
-      <component>
-        <name>STORM_REST_API</name>
-        <category>MASTER</category>
-        <commandScript>
-          <script>scripts/rest_api.py</script>
-          <scriptType>PYTHON</scriptType>
-          <timeout>600</timeout>
-        </commandScript>
-      </component>
-
-      <component>
-        <name>SUPERVISOR</name>
-        <category>SLAVE</category>
-        <commandScript>
-          <script>scripts/supervisor.py</script>
-          <scriptType>PYTHON</scriptType>
-          <timeout>600</timeout>
-        </commandScript>
-      </component>
-
-      <component>
-        <name>STORM_UI_SERVER</name>
-        <category>MASTER</category>
-        <publishConfig>true</publishConfig>
-        <commandScript>
-          <script>scripts/ui_server.py</script>
-          <scriptType>PYTHON</scriptType>
-          <timeout>600</timeout>
-        </commandScript>
-      </component>
-
-      <component>
-        <name>DRPC_SERVER</name>
-        <category>MASTER</category>
-        <commandScript>
-          <script>scripts/drpc_server.py</script>
-          <scriptType>PYTHON</scriptType>
-          <timeout>600</timeout>
-        </commandScript>
-      </component>
-    </components>
-
-    <osSpecifics>
-      <osSpecific>
-        <osType>any</osType>
-        <packages>
-          <package>
-            <type>tarball</type>
-            <name>files/apache-storm-0.9.1.2.1.1.0-237.tar.gz</name>
-          </package>
-        </packages>
-      </osSpecific>
-    </osSpecifics>
-
-    <configuration-dependencies>
-      <config-type>storm-site</config-type>
-      <config-type>global</config-type>
-    </configuration-dependencies>
-  </application>
-</metainfo>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/package/files/apache-storm-0.9.1.2.1.1.0-237.tar.gz.REPLACE
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/package/files/apache-storm-0.9.1.2.1.1.0-237.tar.gz.REPLACE b/app-packages/storm-v0_91/package/files/apache-storm-0.9.1.2.1.1.0-237.tar.gz.REPLACE
deleted file mode 100644
index dd934d5..0000000
--- a/app-packages/storm-v0_91/package/files/apache-storm-0.9.1.2.1.1.0-237.tar.gz.REPLACE
+++ /dev/null
@@ -1,16 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-Replace with the actual storm package.

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/package/scripts/drpc_server.py
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/package/scripts/drpc_server.py b/app-packages/storm-v0_91/package/scripts/drpc_server.py
deleted file mode 100644
index a01d0f0..0000000
--- a/app-packages/storm-v0_91/package/scripts/drpc_server.py
+++ /dev/null
@@ -1,55 +0,0 @@
-#!/usr/bin/env python
-"""
-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.
-
-"""
-
-import sys
-from resource_management import *
-from storm import storm
-from service import service
-
-class DrpcServer(Script):
-  def install(self, env):
-    self.install_packages(env)
-
-  def configure(self, env):
-    import params
-    env.set_params(params)
-
-    storm()
-
-  def start(self, env):
-    import params
-    env.set_params(params)
-    self.configure(env)
-
-    service("drpc", action="start")
-
-  def stop(self, env):
-    import params
-    env.set_params(params)
-
-    service("drpc", action="stop")
-
-  def status(self, env):
-    import status_params
-    env.set_params(status_params)
-    check_process_status(status_params.pid_drpc)
-
-if __name__ == "__main__":
-  DrpcServer().execute()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/package/scripts/nimbus.py
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/package/scripts/nimbus.py b/app-packages/storm-v0_91/package/scripts/nimbus.py
deleted file mode 100644
index c7c3120..0000000
--- a/app-packages/storm-v0_91/package/scripts/nimbus.py
+++ /dev/null
@@ -1,55 +0,0 @@
-#!/usr/bin/env python
-"""
-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.
-
-"""
-
-import sys
-from resource_management import *
-from storm import storm
-from service import service
-
-class Nimbus(Script):
-  def install(self, env):
-    self.install_packages(env)
-
-  def configure(self, env):
-    import params
-    env.set_params(params)
-
-    storm()
-
-  def start(self, env):
-    import params
-    env.set_params(params)
-    self.configure(env)
-
-    service("nimbus", action="start")
-
-  def stop(self, env):
-    import params
-    env.set_params(params)
-
-    service("nimbus", action="stop")
-
-  def status(self, env):
-    import status_params
-    env.set_params(status_params)
-    check_process_status(status_params.pid_nimbus)
-
-if __name__ == "__main__":
-  Nimbus().execute()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/package/scripts/params.py b/app-packages/storm-v0_91/package/scripts/params.py
deleted file mode 100644
index cf21b27..0000000
--- a/app-packages/storm-v0_91/package/scripts/params.py
+++ /dev/null
@@ -1,58 +0,0 @@
-#!/usr/bin/env python
-"""
-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.
-
-"""
-
-from resource_management import *
-import status_params
-
-# server configurations
-config = Script.get_config()
-
-app_root = config['configurations']['global']['app_root']
-conf_dir = format("{app_root}/conf")
-storm_user = config['configurations']['global']['app_user']
-log_dir = config['configurations']['global']['app_log_dir']
-pid_dir = status_params.pid_dir
-local_dir = config['configurations']['storm-site']['storm.local.dir']
-user_group = config['configurations']['global']['user_group']
-java64_home = config['hostLevelParams']['java_home']
-nimbus_host = config['configurations']['storm-site']['nimbus.host']
-nimbus_port = config['configurations']['storm-site']['nimbus.thrift.port']
-nimbus_host = config['configurations']['storm-site']['nimbus.host']
-rest_api_port = config['configurations']['global']['rest_api_port']
-rest_api_admin_port = config['configurations']['global']['rest_api_admin_port']
-rest_api_conf_file = format("{conf_dir}/config.yaml")
-rest_lib_dir = format("{app_root}/contrib/storm-rest")
-storm_bin = format("{app_root}/bin/storm")
-
-ganglia_installed = config['configurations']['global']['ganglia_enabled']
-if ganglia_installed:
-  ganglia_report_interval = 60
-  ganglia_server = config['configurations']['global']['ganglia_server_host']
-  ganglia_port = config['configurations']['global']['ganglia_server_port']
-
-_authentication = config['configurations']['core-site']['hadoop.security.authentication']
-security_enabled = ( not is_empty(_authentication) and _authentication == 'kerberos')
-
-if security_enabled:
-  _hostname_lowercase = config['hostname'].lower()
-  _kerberos_domain = config['configurations']['global']['kerberos_domain']
-  _storm_principal_name = config['configurations']['global']['storm_principal_name']
-  storm_jaas_principal = _storm_principal_name.replace('_HOST', _hostname_lowercase)
-  storm_keytab_path = config['configurations']['global']['storm_keytab']

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/package/scripts/rest_api.py
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/package/scripts/rest_api.py b/app-packages/storm-v0_91/package/scripts/rest_api.py
deleted file mode 100644
index 33d8924..0000000
--- a/app-packages/storm-v0_91/package/scripts/rest_api.py
+++ /dev/null
@@ -1,57 +0,0 @@
-#!/usr/bin/env python
-"""
-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.
-
-"""
-
-import sys
-from resource_management import *
-from storm import storm
-from service import service
-
-
-class StormRestApi(Script):
-  def install(self, env):
-    self.install_packages(env)
-    self.configure(env)
-
-  def configure(self, env):
-    import params
-    env.set_params(params)
-
-    storm()
-
-  def start(self, env):
-    import params
-    env.set_params(params)
-    self.configure(env)
-
-    service("rest_api", action="start")
-
-  def stop(self, env):
-    import params
-    env.set_params(params)
-
-    service("rest_api", action="stop")
-
-  def status(self, env):
-    import status_params
-    env.set_params(status_params)
-    check_process_status(status_params.pid_rest_api)
-
-if __name__ == "__main__":
-  StormRestApi().execute()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/package/scripts/service.py
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/package/scripts/service.py b/app-packages/storm-v0_91/package/scripts/service.py
deleted file mode 100644
index 10fa5b9..0000000
--- a/app-packages/storm-v0_91/package/scripts/service.py
+++ /dev/null
@@ -1,95 +0,0 @@
-#!/usr/bin/env python
-"""
-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.
-
-"""
-
-
-from resource_management import *
-import time
-
-
-def service(
-    name,
-    action='start'):
-  import params
-  import status_params
-
-  pid_file = status_params.pid_files[name]
-  no_op_test = format("ls {pid_file} >/dev/null 2>&1 && ps `cat {pid_file}` >/dev/null 2>&1")
-  jps_path = format("{java64_home}/bin/jps")
-  grep_and_awk = "| grep -v grep | awk '{print $1}'"
-
-  if name == 'ui':
-    #process_cmd = "^java.+backtype.storm.ui.core$"
-    pid_chk_cmd = format("{jps_path} -vl | grep \"^[0-9 ]*backtype.storm.ui.core\" {grep_and_awk}  > {pid_file}")
-  elif name == "rest_api":
-    process_cmd = format("{java64_home}/bin/java -jar {rest_lib_dir}/`ls {rest_lib_dir} | grep -wE storm-rest-[0-9.-]+\.jar` server")
-    crt_pid_cmd = format("pgrep -f \"{process_cmd}\" && pgrep -f \"{process_cmd}\" > {pid_file}")
-  else:
-    #process_cmd = format("^java.+backtype.storm.daemon.{name}$")
-    pid_chk_cmd = format("{jps_path} -vl | grep \"^[0-9 ]*backtype.storm.daemon.{name}\" {grep_and_awk}  > {pid_file}")
-
-  if action == "start":
-    if name == "rest_api":
-      cmd = format("{process_cmd} {rest_api_conf_file} > {log_dir}/restapi.log")
-    else:
-      cmd = format("env JAVA_HOME={java64_home} PATH=$PATH:{java64_home}/bin STORM_BASE_DIR={app_root} STORM_CONF_DIR={conf_dir} {storm_bin} {name}")
-
-    Execute(cmd,
-            not_if=no_op_test,
-            user=params.storm_user,
-            logoutput=False,
-            wait_for_finish=False
-    )
-
-    if name == "rest_api":
-      Execute(crt_pid_cmd,
-              user=params.storm_user,
-              logoutput=True,
-              tries=6,
-              try_sleep=10
-      )
-    else:
-      content = None
-      for i in xrange(12):
-        Execute(pid_chk_cmd,
-                user=params.storm_user,
-                logoutput=True
-        )
-        with open(pid_file) as f:
-          content = f.readline().strip()
-        if content.isdigit():
-          break;
-        File(pid_file, action = "delete")
-        time.sleep(10)
-        pass
-
-      if not content.isdigit():
-        raise Fail(format("Unable to start {name}"))
-
-  elif action == "stop":
-    process_dont_exist = format("! ({no_op_test})")
-    pid = format("`cat {pid_file}` >/dev/null 2>&1")
-    Execute(format("kill {pid}"),
-            not_if=process_dont_exist
-    )
-    Execute(format("kill -9 {pid}"),
-            not_if=format("sleep 2; {process_dont_exist} || sleep 20; {process_dont_exist}"),
-            ignore_failures=True
-    )
-    Execute(format("rm -f {pid_file}"))

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/package/scripts/status_params.py
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/package/scripts/status_params.py b/app-packages/storm-v0_91/package/scripts/status_params.py
deleted file mode 100644
index eab83cf..0000000
--- a/app-packages/storm-v0_91/package/scripts/status_params.py
+++ /dev/null
@@ -1,36 +0,0 @@
-#!/usr/bin/env python
-"""
-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.
-
-"""
-from resource_management import *
-
-config = Script.get_config()
-
-pid_dir = config['configurations']['global']['app_pid_dir']
-pid_nimbus = format("{pid_dir}/nimbus.pid")
-pid_supervisor = format("{pid_dir}/supervisor.pid")
-pid_drpc = format("{pid_dir}/drpc.pid")
-pid_ui = format("{pid_dir}/ui.pid")
-pid_logviewer = format("{pid_dir}/logviewer.pid")
-pid_rest_api = format("{pid_dir}/restapi.pid")
-pid_files = {"logviewer":pid_logviewer,
-             "ui": pid_ui,
-             "nimbus": pid_nimbus,
-             "supervisor": pid_supervisor,
-             "drpc": pid_drpc,
-             "rest_api": pid_rest_api}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/package/scripts/storm.py
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/package/scripts/storm.py b/app-packages/storm-v0_91/package/scripts/storm.py
deleted file mode 100644
index bce272b..0000000
--- a/app-packages/storm-v0_91/package/scripts/storm.py
+++ /dev/null
@@ -1,50 +0,0 @@
-#!/usr/bin/env python
-"""
-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.
-
-"""
-
-from resource_management import *
-from yaml_config import yaml_config
-import sys
-
-def storm():
-  import params
-
-  Directory([params.log_dir, params.pid_dir, params.local_dir, params.conf_dir],
-            owner=params.storm_user,
-            group=params.user_group,
-            recursive=True
-  )
-
-  File(format("{conf_dir}/config.yaml"),
-            content=Template("config.yaml.j2"),
-            owner = params.storm_user,
-            group = params.user_group
-  )
-
-  yaml_config( "storm.yaml",
-               conf_dir = params.conf_dir,
-               configurations = params.config['configurations']['storm-site'],
-               owner = params.storm_user,
-               group = params.user_group
-  )
-  
-  if params.security_enabled:
-    TemplateConfig( format("{conf_dir}/storm_jaas.conf"),
-      owner = params.storm_user
-    )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/package/scripts/supervisor.py
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/package/scripts/supervisor.py b/app-packages/storm-v0_91/package/scripts/supervisor.py
deleted file mode 100644
index 47c20c9..0000000
--- a/app-packages/storm-v0_91/package/scripts/supervisor.py
+++ /dev/null
@@ -1,61 +0,0 @@
-#!/usr/bin/env python
-"""
-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.
-
-"""
-
-import sys
-from resource_management import *
-from yaml_config import yaml_config
-from storm import storm
-from service import service
-
-
-class Supervisor(Script):
-  def install(self, env):
-    self.install_packages(env)
-
-  def configure(self, env):
-    import params
-    env.set_params(params)
-    storm()
-
-  def start(self, env):
-    import params
-    env.set_params(params)
-    self.configure(env)
-
-    service("supervisor", action="start")
-    service("logviewer", action="start")
-
-  def stop(self, env):
-    import params
-    env.set_params(params)
-
-    service("supervisor", action="stop")
-    service("logviewer", action="stop")
-
-  def status(self, env):
-    import status_params
-    env.set_params(status_params)
-
-    check_process_status(status_params.pid_supervisor)
-
-
-if __name__ == "__main__":
-  Supervisor().execute()
-

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/package/scripts/ui_server.py
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/package/scripts/ui_server.py b/app-packages/storm-v0_91/package/scripts/ui_server.py
deleted file mode 100644
index 0fe7cd2..0000000
--- a/app-packages/storm-v0_91/package/scripts/ui_server.py
+++ /dev/null
@@ -1,55 +0,0 @@
-#!/usr/bin/env python
-"""
-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.
-
-"""
-
-import sys
-from resource_management import *
-from storm import storm
-from service import service
-
-class UiServer(Script):
-  def install(self, env):
-    self.install_packages(env)
-
-  def configure(self, env):
-    import params
-    env.set_params(params)
-
-    storm()
-
-  def start(self, env):
-    import params
-    env.set_params(params)
-    self.configure(env)
-
-    service("ui", action="start")
-
-  def stop(self, env):
-    import params
-    env.set_params(params)
-
-    service("ui", action="stop")
-
-  def status(self, env):
-    import status_params
-    env.set_params(status_params)
-    check_process_status(status_params.pid_ui)
-
-if __name__ == "__main__":
-  UiServer().execute()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/package/scripts/yaml_config.py
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/package/scripts/yaml_config.py b/app-packages/storm-v0_91/package/scripts/yaml_config.py
deleted file mode 100644
index 39261be..0000000
--- a/app-packages/storm-v0_91/package/scripts/yaml_config.py
+++ /dev/null
@@ -1,69 +0,0 @@
-#!/usr/bin/env python
-"""
-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.
-
-"""
-
-import re
-from resource_management import *
-
-def escape_yaml_propetry(value):
-  unquouted = False
-  unquouted_values = ["null","Null","NULL","true","True","TRUE","false","False","FALSE","YES","Yes","yes","NO","No","no","ON","On","on","OFF","Off","off"]
-  
-  if value in unquouted_values:
-    unquouted = True
-
-  # if is list [a,b,c]
-  if re.match('^\w*\[.+\]\w*$', value):
-    unquouted = True
-    
-  try:
-    int(value)
-    unquouted = True
-  except ValueError:
-    pass
-  
-  try:
-    float(value)
-    unquouted = True
-  except ValueError:
-    pass
-  
-  if not unquouted:
-    value = value.replace("'","''")
-    value = "'"+value+"'"
-    
-  return value
-
-def yaml_config(
-  filename,
-  configurations = None,
-  conf_dir = None,
-  mode = None,
-  owner = None,
-  group = None
-):
-    config_content = source.InlineTemplate('''{% for key, value in configurations_dict.items() %}{{ key }}: {{ escape_yaml_propetry(value) }}
-{% endfor %}''', configurations_dict=configurations, extra_imports=[escape_yaml_propetry])
-
-    File (format("{conf_dir}/{filename}"),
-      content = config_content,
-      owner = owner,
-      group = group,
-      mode = mode
-    )

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/package/templates/config.yaml.j2
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/package/templates/config.yaml.j2 b/app-packages/storm-v0_91/package/templates/config.yaml.j2
deleted file mode 100644
index 32d2c99..0000000
--- a/app-packages/storm-v0_91/package/templates/config.yaml.j2
+++ /dev/null
@@ -1,48 +0,0 @@
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-
-nimbusHost: {{nimbus_host}}
-nimbusPort: {{nimbus_port}}
-
-# HTTP-specific options.
-http:
-
-  # The port on which the HTTP server listens for service requests.
-  port: {{rest_api_port}}
-
-  # The port on which the HTTP server listens for administrative requests.
-  adminPort: {{rest_api_admin_port}}
-
-{% if ganglia_installed %}
-enableGanglia: {{ganglia_installed}}
-
-# ganglia configuration (necessary if ganglia reporting is enabled)
-ganglia:
-
-  # how often to report to ganglia metrics (in seconds)
-  reportInterval: {{ganglia_report_interval}}
-
-  # the hostname of the gmond server where storm cluster metrics will be sent
-  host: {{ganglia_server}}
-  port: {{ganglia_port}}
-
-  # address mode
-  # default is MULTICAST
-  addressMode: "UNICAST"
-
-  # an <IP>:<HOSTNAME> pair to spoof
-  # this allows us to simulate storm cluster metrics coming from a specific host
-  #spoof: "192.168.1.1:storm"
-{% endif %}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/package/templates/storm_jaas.conf.j2
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/package/templates/storm_jaas.conf.j2 b/app-packages/storm-v0_91/package/templates/storm_jaas.conf.j2
deleted file mode 100644
index 4031d22..0000000
--- a/app-packages/storm-v0_91/package/templates/storm_jaas.conf.j2
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- *  or more contributor license agreements.  See the NOTICE file
- *  distributed with this work for additional information
- *  regarding copyright ownership.  The ASF licenses this file
- *  to you under the Apache License, Version 2.0 (the
- *  "License"); you may not use this file except in compliance
- *  with the License.  You may obtain a copy of the License at
- *
- *       http://www.apache.org/licenses/LICENSE-2.0
- *
- *  Unless required by applicable law or agreed to in writing, software
- *  distributed under the License is distributed on an "AS IS" BASIS,
- *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- *  See the License for the specific language governing permissions and
- *  limitations under the License.
- */
-Client {
-   com.sun.security.auth.module.Krb5LoginModule required
-   useKeyTab=true
-   keyTab="{{storm_keytab_path}}"
-   storeKey=true
-   useTicketCache=false
-   serviceName="zookeeper"
-   principal="{{storm_jaas_principal}}";
-};

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm-v0_91/resources.json
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/resources.json b/app-packages/storm-v0_91/resources.json
deleted file mode 100644
index b184a40..0000000
--- a/app-packages/storm-v0_91/resources.json
+++ /dev/null
@@ -1,31 +0,0 @@
-{
-  "schema" : "http://example.org/specification/v2.0.0",
-  "metadata" : {
-  },
-  "global" : {
-  },
-  "components": {
-    "slider-appmaster": {
-    },
-    "NIMBUS": {
-      "yarn.role.priority": "1",
-      "yarn.component.instances": "1"
-    },
-    "STORM_REST_API": {
-      "yarn.role.priority": "2",
-      "yarn.component.instances": "1"
-    },
-    "STORM_UI_SERVER": {
-      "yarn.role.priority": "3",
-      "yarn.component.instances": "1"
-    },
-    "DRPC_SERVER": {
-      "yarn.role.priority": "4",
-      "yarn.component.instances": "1"
-    },
-    "SUPERVISOR": {
-      "yarn.role.priority": "5",
-      "yarn.component.instances": "1"
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/README.txt
----------------------------------------------------------------------
diff --git a/app-packages/storm/README.txt b/app-packages/storm/README.txt
new file mode 100644
index 0000000..971cf14
--- /dev/null
+++ b/app-packages/storm/README.txt
@@ -0,0 +1,37 @@
+<!---
+   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.
+-->
+
+How to create a Slider app package for Storm?
+
+To create the app package you will need the Storm tarball copied to a specific location.
+Various configurations provided in this sample are customized for apache-storm-0.9.1.2.1.1.0-237.tar.gz.
+So if you use a different version you may need to edit a few config values.
+
+Replace the placeholder tarball for Storm.
+  cp ~/Downloads/apache-storm-0.9.1.2.1.1.0-237.tar.gz package/files/
+  rm package/files/apache-storm-0.9.1.2.1.1.0-237.tar.gz.REPLACE
+
+Create a zip package at the root of the package (<slider enlistment>/app-packages/storm-v0_91/) 
+  zip -r storm_v091.zip .
+
+Verify the content using  
+  unzip -l "$@" storm_v091.zip
+
+While appConfig.json and resources.json are not required for the package they work
+well as the default configuration for Slider apps. So its advisable that when you
+create an application package for Slider, include sample/default resources.json and
+appConfig.json for a minimal Yarn cluster.


[21/50] [abbrv] git commit: SLIDER-215 Addendum fixes typo of KEY_TEST_HBASE_LAUNCH_TIME

Posted by sm...@apache.org.
SLIDER-215 Addendum fixes typo of KEY_TEST_HBASE_LAUNCH_TIME


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

Branch: refs/heads/master
Commit: f50ea0e989e9ed96caa802065407a0440835be37
Parents: da59d88
Author: tedyu <yu...@gmail.com>
Authored: Mon Jul 7 15:44:35 2014 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Jul 7 15:44:35 2014 -0700

----------------------------------------------------------------------
 .../apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f50ea0e9/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy b/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy
index fa2ac49..757b7fa 100644
--- a/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy
+++ b/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy
@@ -34,7 +34,7 @@ abstract class HBaseAgentCommandTestBase extends AgentCommandTestBase {
 
   static {
     HBASE_LAUNCH_WAIT_TIME = getTimeOptionMillis(SLIDER_CONFIG,
-      KEY_HBASE_LAUNCH_TIME,
+      KEY_TEST_HBASE_LAUNCH_TIME,
       1000 * DEFAULT_HBASE_LAUNCH_TIME_SECONDS)
   }
 


[35/50] [abbrv] git commit: Merge branch 'feature/SLIDER-221_Create_an_RPM_for_slider' into develop

Posted by sm...@apache.org.
Merge branch 'feature/SLIDER-221_Create_an_RPM_for_slider' into develop


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

Branch: refs/heads/master
Commit: 729b5623903e0ef3cb971cdcde032c1641d27c8d
Parents: 3427b6e 27fe84d
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 9 17:04:48 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 9 17:04:48 2014 +0100

----------------------------------------------------------------------
 pom.xml                                        |   2 +
 slider-assembly/pom.xml                        | 183 +++++++++++++++++++-
 slider-assembly/src/conf-hdp/log4j.properties  |  58 +++++++
 slider-assembly/src/conf-hdp/slider-client.xml |  77 ++++++++
 slider-assembly/src/conf/slider-client.xml     |   3 -
 slider-install/pom.xml                         | 111 ++++++++++++
 6 files changed, 426 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[40/50] [abbrv] git commit: SLIDER-220 Provide script for retrieving hbase-site.xml given cluster name

Posted by sm...@apache.org.
SLIDER-220 Provide script for retrieving hbase-site.xml given cluster name


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

Branch: refs/heads/master
Commit: 80424c1d57328081fcb37d4a6d17f9836206f61c
Parents: 55aa2b1
Author: tedyu <yu...@gmail.com>
Authored: Wed Jul 9 12:53:23 2014 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Jul 9 12:53:23 2014 -0700

----------------------------------------------------------------------
 app-packages/hbase/get-hbase-site.sh | 24 ++++++++++++++++++++++++
 get-hbase-site.sh                    |  8 --------
 2 files changed, 24 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/80424c1d/app-packages/hbase/get-hbase-site.sh
----------------------------------------------------------------------
diff --git a/app-packages/hbase/get-hbase-site.sh b/app-packages/hbase/get-hbase-site.sh
new file mode 100755
index 0000000..0edac30
--- /dev/null
+++ b/app-packages/hbase/get-hbase-site.sh
@@ -0,0 +1,24 @@
+# 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.
+
+tuple=`slider status $1 | grep "info.am.web.url"`
+FS=":"
+url=`echo $tuple | awk '{split($0,array,": ")} END{print array[2]}'`
+url="${url%,}"
+url="${url%\"}"
+url="${url#\"}"
+url="${url}ws/v1/slider/publisher/slider/hbase-site.xml"
+curl -k -o hbase-site.xml $url

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/80424c1d/get-hbase-site.sh
----------------------------------------------------------------------
diff --git a/get-hbase-site.sh b/get-hbase-site.sh
deleted file mode 100755
index 32942a2..0000000
--- a/get-hbase-site.sh
+++ /dev/null
@@ -1,8 +0,0 @@
-tuple=`slider status $1 | grep "info.am.web.url"`
-FS=":"
-url=`echo $tuple | awk '{split($0,array,": ")} END{print array[2]}'`
-url="${url%,}"
-url="${url%\"}"
-url="${url#\"}"
-url="${url}ws/v1/slider/publisher/slider/hbase-site.xml"
-curl -k -o hbase-site.xml $url


[03/50] [abbrv] git commit: SLIDER-199. Modify HBase package README to also add a manual way to create .zip packages

Posted by sm...@apache.org.
SLIDER-199. Modify HBase package README to also add a manual way to create .zip packages


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

Branch: refs/heads/master
Commit: 5bb80a42b1f5d6ea0c37d54b9dcb3f2ebf9251e1
Parents: e4723f5
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Tue Jul 1 17:42:43 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Tue Jul 1 17:42:48 2014 -0700

----------------------------------------------------------------------
 app-packages/hbase/README.txt | 54 +++++++++++++++++++++++++-------------
 pom.xml                       |  1 -
 2 files changed, 36 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5bb80a42/app-packages/hbase/README.txt
----------------------------------------------------------------------
diff --git a/app-packages/hbase/README.txt b/app-packages/hbase/README.txt
index a25ff8c..1d547a2 100644
--- a/app-packages/hbase/README.txt
+++ b/app-packages/hbase/README.txt
@@ -15,40 +15,58 @@
    limitations under the License.
 -->
 
-How to create a Slider package for HBase?
+Create Slider App Package for HBase
+
+While appConfig.json and resources.json are not required for the package they
+work well as the default configuration for Slider apps. So it is advisable that
+when you create an application package for Slider, include sample/default
+resources.json and appConfig.json for a minimal Yarn cluster.
+
+OPTION-I: Use mvn command
+OPTION-II: Manual
+
+****** OPTION - I (use mvn command) **
+You need the HBase version available on local maven repo to create the Slider App Package for HBase.
 
 The version of HBase used for the app package can be adjusted by adding a
 flag such as
   -Dhbase.version=0.98.3
 
-Copy the tarball for HBase:
-  cp ~/Downloads/hbase-0.98.3-hadoop2-bin.tar.gz package/files/
+Download the tarball for HBase:
+  e.g. path to tarball ~/Downloads/hbase-0.98.3-hadoop2-bin.tar.gz
 
 Use the following command to install HBase tarball locally:
   mvn install:install-file -Dfile=<path-to-tarball> -DgroupId=org.apache.hbase -DartifactId=hbase -Dversion=0.98.3-hadoop2 -Dclassifier=bin -Dpackaging=tar.gz
 
-After HBase tarball is publised locally in maven repository, you can use the following command:
+After HBase tarball is published locally in maven repository, you can use the following command:
   mvn clean package -DskipTests -Phbase-app-package
+
 App package can be found in
   app-packages/hbase/target/apache-slider-hbase-${hbase.version}-app-package-${slider.version}.zip
 
-Create a zip package at the root of the package (<slider enlistment>/app-packages/hbase/)
-  zip -r hbase-v098.zip .
-
 Verify the content using
   zip -Tv apache-slider-hbase-*.zip
 
-While appConfig.json and resources.json are not required for the package they
-work well as the default configuration for Slider apps. So it is advisable that
-when you create an application package for Slider, include sample/default
-resources.json and appConfig.json for a minimal Yarn cluster.
+If an HBase version older than 0.98.3 is desired, it must be installed in the local maven repo.
 
-If an HBase version older than 0.98.3 is desired, it must be installed in the
-local maven repo.
+A less descriptive file name can be specified with
+  -Dapp.package.name=HBase_98dot3 which would create a file HBase_98dot3.zip.
 
-**Note that the LICENSE.txt and NOTICE.txt that are bundled with the app
-package are designed for HBase 0.98.3 only and may need to be modified to be
-applicable for other versions of the app package.
+****** OPTION - II (manual) **
+The Slider App Package for HBase can also be created manually.
 
-A less descriptive file name can be specified with
--Dapp.package.name=HBase_98dot3 which would create a file HBase_98dot3.zip.
+Download the tarball for HBase:
+  e.g. path to tarball ~/Downloads/hbase-0.98.3-hadoop2-bin.tar.gz
+
+Copy the hbase tarball to package/files
+  cp ~/Downloads/hbase-0.98.3-hadoop2-bin.tar.gz package/files
+
+Edit appConfig.json/metainfo.xml
+  Replace 4 occurrences of "${hbase.version}" with the hbase version values such as "0.98.3-hadoop2"
+  Replace 1 occurrence of "${app.package.name}" with the desired app package name, e.g. "hbase-v098"
+
+Create a zip package at the root of the package (<slider enlistment>/app-packages/hbase/)
+  zip -r hbase-v098.zip .
+
+Verify the content using
+  zip -Tv hbase-v098.zip

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/5bb80a42/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e1f772b..460fb0d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -39,7 +39,6 @@
     <module>slider-core</module>
     <module>slider-agent</module>
     <module>app-packages/accumulo</module>
-    <module>app-packages/hbase</module>
     <module>slider-assembly</module>
     <module>slider-funtest</module>
     <module>slider-providers/hbase/slider-hbase-provider</module>


[37/50] [abbrv] git commit: SLIDER-220 Provide script for retrieving hbase-site.xml given cluster name

Posted by sm...@apache.org.
SLIDER-220 Provide script for retrieving hbase-site.xml given cluster name


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

Branch: refs/heads/master
Commit: 55aa2b1f6bbb2f34883e392c9df50f8c68559121
Parents: c1a5305
Author: tedyu <yu...@gmail.com>
Authored: Wed Jul 9 11:36:35 2014 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Jul 9 11:36:35 2014 -0700

----------------------------------------------------------------------
 get-hbase-site.sh | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/55aa2b1f/get-hbase-site.sh
----------------------------------------------------------------------
diff --git a/get-hbase-site.sh b/get-hbase-site.sh
new file mode 100755
index 0000000..32942a2
--- /dev/null
+++ b/get-hbase-site.sh
@@ -0,0 +1,8 @@
+tuple=`slider status $1 | grep "info.am.web.url"`
+FS=":"
+url=`echo $tuple | awk '{split($0,array,": ")} END{print array[2]}'`
+url="${url%,}"
+url="${url%\"}"
+url="${url#\"}"
+url="${url}ws/v1/slider/publisher/slider/hbase-site.xml"
+curl -k -o hbase-site.xml $url


[04/50] [abbrv] git commit: SLIDER-159 TestLongLivedProcess.testEcho failing

Posted by sm...@apache.org.
SLIDER-159 TestLongLivedProcess.testEcho failing


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

Branch: refs/heads/master
Commit: e9ddcf0b13beb6aea3822ee0164f6f46f40676f2
Parents: 5bb80a4
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 2 13:23:54 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 2 13:23:54 2014 +0100

----------------------------------------------------------------------
 .../services/workflow/ForkedProcessService.java | 22 +-------------
 .../services/workflow/LongLivedProcess.java     | 31 ++++++++++++++++++++
 .../services/workflow/TestLongLivedProcess.java |  5 ++--
 3 files changed, 35 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e9ddcf0b/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java b/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
index ee68aa4..ccce6cb 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/workflow/ForkedProcessService.java
@@ -279,27 +279,7 @@ public class ForkedProcessService extends AbstractWorkflowExecutorService implem
     if (process == null) {
       return new LinkedList<String>();
     }
-    long start = System.currentTimeMillis();
-    while (System.currentTimeMillis() - start <= duration) {
-      boolean finished;
-      if (finalOutput) {
-        // final flag means block until all data is done
-        finished = process.isFinalOutputProcessed();
-      } else {
-        // there is some output
-        finished = !process.isRecentOutputEmpty();
-      }
-      if (finished) {
-        break;
-      }
-      try {
-        Thread.sleep(100);
-      } catch (InterruptedException ie) {
-        Thread.currentThread().interrupt();
-        break;
-      }
-    }
-    return process.getRecentOutput();
+    return process.getRecentOutput(finalOutput, duration);
   }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e9ddcf0b/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java b/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
index d9ddecb..05a1c50 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
@@ -357,6 +357,37 @@ public class LongLivedProcess implements Runnable {
   }
 
   /**
+   * Get the recent output from the process, or [] if not defined
+   *
+   * @param finalOutput flag to indicate "wait for the final output of the process"
+   * @param duration the duration, in ms, 
+   * ro wait for recent output to become non-empty
+   * @return a possibly empty list
+   */
+  public List<String> getRecentOutput(boolean finalOutput, int duration) {
+    long start = System.currentTimeMillis();
+    while (System.currentTimeMillis() - start <= duration) {
+      boolean finishedOutput;
+      if (finalOutput) {
+        // final flag means block until all data is done
+        finishedOutput = isFinalOutputProcessed();
+      } else {
+        // there is some output
+        finishedOutput = !isRecentOutputEmpty();
+      }
+      if (finishedOutput) {
+        break;
+      }
+      try {
+        Thread.sleep(100);
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+        break;
+      }
+    }
+    return getRecentOutput();
+  }
+  /**
    * add the recent line to the list of recent lines; deleting
    * an earlier on if the limit is reached.
    *

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e9ddcf0b/slider-core/src/test/java/org/apache/slider/server/services/workflow/TestLongLivedProcess.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/services/workflow/TestLongLivedProcess.java b/slider-core/src/test/java/org/apache/slider/server/services/workflow/TestLongLivedProcess.java
index c8a0719..668bcca 100644
--- a/slider-core/src/test/java/org/apache/slider/server/services/workflow/TestLongLivedProcess.java
+++ b/slider-core/src/test/java/org/apache/slider/server/services/workflow/TestLongLivedProcess.java
@@ -126,13 +126,14 @@ public class TestLongLivedProcess extends WorkflowServiceTestBase implements
   }
 
   /**
-   * Get the final output. includes a quick sleep for the tail output
+   * Get the final output. 
    * @return the last output
    */
   private List<String> getFinalOutput() {
-    return process.getRecentOutput();
+    return process.getRecentOutput(true, 4000);
   }
 
+
   private LongLivedProcess initProcess(List<String> commands) {
     process = new LongLivedProcess(name.getMethodName(), log, commands);
     process.setLifecycleCallback(this);


[13/50] [abbrv] git commit: Add note for building hbase package

Posted by sm...@apache.org.
Add note for building hbase package


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

Branch: refs/heads/master
Commit: 87fe586158955b7632106bd7ec02d7493a461693
Parents: 0f9344b
Author: tedyu <yu...@gmail.com>
Authored: Mon Jul 7 07:28:21 2014 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Jul 7 07:28:21 2014 -0700

----------------------------------------------------------------------
 app-packages/hbase/README.txt | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/87fe5861/app-packages/hbase/README.txt
----------------------------------------------------------------------
diff --git a/app-packages/hbase/README.txt b/app-packages/hbase/README.txt
index 1d547a2..b4e4ccd 100644
--- a/app-packages/hbase/README.txt
+++ b/app-packages/hbase/README.txt
@@ -38,6 +38,9 @@ Download the tarball for HBase:
 Use the following command to install HBase tarball locally:
   mvn install:install-file -Dfile=<path-to-tarball> -DgroupId=org.apache.hbase -DartifactId=hbase -Dversion=0.98.3-hadoop2 -Dclassifier=bin -Dpackaging=tar.gz
 
+You may need to copy the hbase tarball to the following location if the above step doesn't publish the tarball:
+~/.m2/repository/org/apache/hbase/hbase/0.98.3-hadoop2/
+
 After HBase tarball is published locally in maven repository, you can use the following command:
   mvn clean package -DskipTests -Phbase-app-package
 


[10/50] [abbrv] git commit: SLIDER-207 Remove extraneous character in front of ipc.client.connect.retry.interval

Posted by sm...@apache.org.
SLIDER-207 Remove extraneous character in front of ipc.client.connect.retry.interval


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

Branch: refs/heads/master
Commit: 4933cd8a5b40ec0964986bc61c2ac114b8622311
Parents: f94e103
Author: tedyu <yu...@gmail.com>
Authored: Wed Jul 2 21:07:12 2014 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Jul 2 21:07:12 2014 -0700

----------------------------------------------------------------------
 .../org/apache/slider/funtest/basic/TestClusterConnectivity.groovy | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4933cd8a/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
index dcc7333..8436dc9 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
@@ -60,7 +60,7 @@ class TestClusterConnectivity extends CommandTestBase {
     testRMTelnet()
     SliderYarnClientImpl yarnClient = new SliderYarnClientImpl()
     try {
-      SLIDER_CONFIG.setInt("ipc.client.connect.retry.interval",100)
+      SLIDER_CONFIG.setInt("ipc.client.connect.retry.interval",100)
       SLIDER_CONFIG.setInt(
           YarnConfiguration.RESOURCEMANAGER_CONNECT_MAX_WAIT_MS,5000)
       SLIDER_CONFIG.setInt(


[06/50] [abbrv] git commit: SLIDER-201 jenkins windows support - fail fast on HADOOP-10775

Posted by sm...@apache.org.
SLIDER-201 jenkins windows support - fail fast on  HADOOP-10775


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

Branch: refs/heads/master
Commit: a6cbab17bf591d7b0a6f908ea291c6b4c549bd6d
Parents: 0151b9f
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 2 15:37:54 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 2 15:37:54 2014 +0100

----------------------------------------------------------------------
 .../org/apache/slider/test/YarnMiniClusterTestBase.groovy   | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/a6cbab17/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy b/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
index e5715b1..a5c8710 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.FileUtil
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.hdfs.MiniDFSCluster
 import org.apache.hadoop.service.ServiceOperations
+import org.apache.hadoop.util.Shell
 import org.apache.hadoop.yarn.api.records.ApplicationReport
 import org.apache.hadoop.yarn.api.records.YarnApplicationState
 import org.apache.hadoop.yarn.conf.YarnConfiguration
@@ -48,7 +49,9 @@ import org.apache.slider.core.main.ServiceLauncher
 import org.apache.slider.core.main.ServiceLauncherBaseTest
 import org.apache.slider.server.appmaster.SliderAppMaster
 import org.junit.After
+import org.junit.Assert
 import org.junit.Before
+import org.junit.BeforeClass
 import org.junit.Rule
 import org.junit.rules.TestName
 import org.junit.rules.Timeout
@@ -126,6 +129,12 @@ public abstract class YarnMiniClusterTestBase extends ServiceLauncherBaseTest {
           KEY_TEST_TIMEOUT,
           DEFAULT_TEST_TIMEOUT_SECONDS * 1000)
   )
+  @BeforeClass
+  public void checkWindowsSupport() {
+    if (Shell.WINDOWS) {
+      Assert.assertNotNull("winutils.exe not found", Shell.WINUTILS)
+    }
+  } 
 
 
   @Rule


[08/50] [abbrv] git commit: SLIDER-3 fix typo in listRegisteredInstances() method

Posted by sm...@apache.org.
SLIDER-3 fix typo in listRegisteredInstances() method


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

Branch: refs/heads/master
Commit: 7823d3a20d1e496d20e5a6c0fcd77d1322d63a49
Parents: 6d96ba9
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 2 17:02:48 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 2 17:02:48 2014 +0100

----------------------------------------------------------------------
 .../src/main/java/org/apache/slider/client/SliderClient.java       | 2 +-
 .../apache/slider/agent/standalone/TestStandaloneAgentAM.groovy    | 2 +-
 .../apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy | 2 +-
 .../providers/hbase/minicluster/live/TestTwoLiveClusters.groovy    | 2 +-
 4 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/7823d3a2/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 a337533..c35a60a 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
@@ -2192,7 +2192,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    * @throws IOException
    * @throws YarnException
    */
-  public List<String> listRegistedSliderInstances() throws
+  public List<String> listRegisteredSliderInstances() throws
       IOException,
       YarnException {
     try {

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/7823d3a2/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy
index d700fea..bce24e6 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneAgentAM.groovy
@@ -101,7 +101,7 @@ class TestStandaloneAgentAM  extends AgentMiniClusterTestBase {
     dumpRegistryServiceTypes(names)
     describe "service registry instance IDs"
 
-    def instanceIds = client.listRegistedSliderInstances()
+    def instanceIds = client.listRegisteredSliderInstances()
 
     log.info("number of instanceIds: ${instanceIds.size()}")
     instanceIds.each { String it -> log.info(it) }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/7823d3a2/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy
index 7639375..cb55624 100644
--- a/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/agent/standalone/TestStandaloneRegistryAM.groovy
@@ -117,7 +117,7 @@ class TestStandaloneRegistryAM extends AgentMiniClusterTestBase {
     def serviceTypes = registryService.serviceTypes;
     dumpRegistryServiceTypes(serviceTypes)
 
-    List<String> instanceIds = client.listRegistedSliderInstances()
+    List<String> instanceIds = client.listRegisteredSliderInstances()
 
 
     dumpRegistryInstanceIDs(instanceIds)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/7823d3a2/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestTwoLiveClusters.groovy
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestTwoLiveClusters.groovy b/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestTwoLiveClusters.groovy
index a58f99a..0140030 100644
--- a/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestTwoLiveClusters.groovy
+++ b/slider-providers/hbase/slider-hbase-provider/src/test/groovy/org/apache/slider/providers/hbase/minicluster/live/TestTwoLiveClusters.groovy
@@ -88,7 +88,7 @@ class TestTwoLiveClusters extends HBaseMiniClusterTestBase {
     def names = registry.getServiceTypes();
     dumpRegistryServiceTypes(names)
 
-    List<String> instanceIds = sliderClient.listRegistedSliderInstances()
+    List<String> instanceIds = sliderClient.listRegisteredSliderInstances()
 
 
     dumpRegistryInstanceIDs(instanceIds)


[24/50] [abbrv] git commit: SLIDER-217 Slider app-package for hbase is not generating the zip file

Posted by sm...@apache.org.
SLIDER-217 Slider app-package for hbase is not generating the zip file


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

Branch: refs/heads/master
Commit: c305ce979459d8e61f53688124ead7ac1cb18ec9
Parents: 0007cb2
Author: tedyu <yu...@gmail.com>
Authored: Tue Jul 8 08:44:18 2014 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Jul 8 08:44:18 2014 -0700

----------------------------------------------------------------------
 app-packages/hbase/src/assembly/hbase.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/c305ce97/app-packages/hbase/src/assembly/hbase.xml
----------------------------------------------------------------------
diff --git a/app-packages/hbase/src/assembly/hbase.xml b/app-packages/hbase/src/assembly/hbase.xml
index 00b69ff..ff1c395 100644
--- a/app-packages/hbase/src/assembly/hbase.xml
+++ b/app-packages/hbase/src/assembly/hbase.xml
@@ -24,6 +24,7 @@
   <id>hbase_v${hbase.version}</id>
   <formats>
     <format>zip</format>
+    <format>dir</format>
   </formats>
   <includeBaseDirectory>false</includeBaseDirectory>
 


[30/50] [abbrv] git commit: SLIDER-221 stub POM with example RPM Config pasted in off the example on the mvn plugin site

Posted by sm...@apache.org.
SLIDER-221 stub POM with example RPM Config pasted in off the example on the mvn plugin site


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

Branch: refs/heads/master
Commit: 497d11c35a51a5a791f65d9a4b3d7940d9a7930d
Parents: 3427b6e
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 9 12:27:40 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 9 12:27:40 2014 +0100

----------------------------------------------------------------------
 pom.xml                |   2 +
 slider-install/pom.xml | 226 ++++++++++++++++++++++++++++++++++++++++++++
 2 files changed, 228 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/497d11c3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 49e8ce7..1ee78fe 100644
--- a/pom.xml
+++ b/pom.xml
@@ -45,6 +45,7 @@
     <module>slider-providers/hbase/hbase-funtests</module>
     <module>slider-providers/accumulo/slider-accumulo-provider</module>
     <module>slider-providers/accumulo/accumulo-funtests</module>
+    <module>slider-install</module>
   </modules>
 
   <licenses>
@@ -179,6 +180,7 @@
     <maven.project.version>2.4</maven.project.version>
     <maven.properties.version>1.0-alpha-2</maven.properties.version>
     <maven-project-info-reports-plugin.version>2.7</maven-project-info-reports-plugin.version>
+    <maven-rpm-plugin.version>2.1-alpha-4</maven-rpm-plugin.version>
     <maven-site-plugin.version>3.3</maven-site-plugin.version>
     <maven-source-plugin.version>2.2.1</maven-source-plugin.version>
     <maven-surefire-plugin.version>2.16</maven-surefire-plugin.version>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/497d11c3/slider-install/pom.xml
----------------------------------------------------------------------
diff --git a/slider-install/pom.xml b/slider-install/pom.xml
new file mode 100644
index 0000000..a00dca9
--- /dev/null
+++ b/slider-install/pom.xml
@@ -0,0 +1,226 @@
+<!--
+   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.
+-->
+<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">
+  <modelVersion>4.0.0</modelVersion>
+  <artifactId>slider-install</artifactId>
+  <name>Slider Install</name>
+  <packaging>pom</packaging>
+  <description>
+    
+    Builds any RPMs and other install packaging for Slider.
+    This is downstream of the slider assembly and the app packages, so
+    that anything from these can be installed in the RPM.
+  </description>
+  <parent>
+    <groupId>org.apache.slider</groupId>
+    <artifactId>slider</artifactId>
+    <version>0.31.0-incubating-SNAPSHOT</version>
+  </parent>
+
+  <build>
+    <plugins>
+      <!--read in a build.properties file if defined-->
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>properties-maven-plugin</artifactId>
+        <version>${maven.properties.version}</version>
+        <executions>
+          <execution>
+            <phase>initialize</phase>
+            <goals>
+              <goal>read-project-properties</goal>
+            </goals>
+            <configuration>
+              <quiet>true</quiet>
+              <files>
+                <file>build.properties</file>
+                <file>../build.properties</file>
+              </files>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+
+
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>rpm-maven-plugin</artifactId>
+        <version>${maven-rpm-plugin.version}</version>
+        <executions>
+          <execution>
+            <id>generate-rpm</id>
+            <goals>
+              <goal>rpm</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <license>http://www.apache.org/licenses/LICENSE-2.0</license>
+          <distribution>Trash 2005</distribution>
+          <group>Application/Collectors</group>
+          <icon>src/main/resources/icon.gif</icon>
+          <packager>SWWDC</packager>
+          <prefix>/usr/local</prefix>
+          <changelogFile>src/changelog</changelogFile>
+          <defineStatements>
+            <defineStatement>_unpackaged_files_terminate_build 0</defineStatement>
+          </defineStatements>
+          <mappings>
+            <mapping>
+              <directory>/usr/local/bin/landfill</directory>
+              <filemode>440</filemode>
+              <username>dumper</username>
+              <groupname>dumpgroup</groupname>
+              <sources>
+                <source>
+                  <location>target/classes</location>
+                </source>
+              </sources>
+            </mapping>
+            <mapping>
+              <directory>/usr/local/doc/landfill</directory>
+              <documentation>true</documentation>
+              <filemode>444</filemode>
+              <username>dumper</username>
+              <groupname>dumpgroup</groupname>
+              <sources>
+                <source>
+                  <location>target/site</location>
+                </source>
+              </sources>
+            </mapping>
+            <mapping>
+              <directory>/usr/local/lib</directory>
+              <filemode>750</filemode>
+              <username>dumper</username>
+              <groupname>dumpgroup</groupname>
+              <dependency>
+                <includes>
+                  <include>jmock:jmock</include>
+                  <include>javax.servlet:servlet-api:2.4</include>
+                </includes>
+                <excludes>
+                  <exclude>junit:junit</exclude>
+                </excludes>
+              </dependency>
+            </mapping>
+            <mapping>
+              <directory>/usr/local/bin</directory>
+              <filemode>750</filemode>
+              <username>dumper</username>
+              <groupname>dumpgroup</groupname>
+              <sources>
+                <source>
+                  <location>src/main/bin</location>
+                </source>
+                <softlinkSource>
+                  <location>/usr/local/bin/new.sh</location>
+                  <destination>old.sh</destination>
+                </softlinkSource>
+              </sources>
+            </mapping>
+            <mapping>
+              <directory>/usr/local/oldbin</directory>
+              <filemode>750</filemode>
+              <username>dumper</username>
+              <groupname>dumpgroup</groupname>
+              <sources>
+                <softlinkSource>
+                  <location>/usr/local/bin</location>
+                </softlinkSource>
+              </sources>
+            </mapping>
+            <mapping>
+              <directory>/usr/local/conf/landfill</directory>
+              <configuration>true</configuration>
+              <filemode>640</filemode>
+              <username>dumper</username>
+              <groupname>dumpgroup</groupname>
+              <sources>
+                <source>
+                  <location>src/main/conf</location>
+                </source>
+              </sources>
+            </mapping>
+            <mapping>
+              <directory>/usr/local/log/landfill</directory>
+              <filemode>750</filemode>
+              <username>dumper</username>
+              <groupname>dumpgroup</groupname>
+            </mapping>
+          </mappings>
+          <preinstallScriptlet>
+            <script>echo "installing now"</script>
+          </preinstallScriptlet>
+          <postinstallScriptlet>
+            <scriptFile>src/main/scripts/postinstall</scriptFile>
+            <fileEncoding>utf-8</fileEncoding>
+          </postinstallScriptlet>
+          <preremoveScriptlet>
+            <scriptFile>src/main/scripts/preremove</scriptFile>
+            <fileEncoding>utf-8</fileEncoding>
+          </preremoveScriptlet>
+        </configuration>
+      </plugin>
+    </plugins>
+    
+    
+  </build>
+
+  <reporting>
+    <plugins>
+ 
+
+
+    </plugins>
+  </reporting>
+
+  <dependencies>
+
+    <dependency>
+      <groupId>org.apache.slider</groupId>
+      <artifactId>slider-core</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+
+    <!--
+     needed to order the build and ensure the agent tar is found
+     the test scope ensures that it isn't copied into the lib dir
+     -->
+    <dependency>
+      <groupId>org.apache.slider</groupId>
+      <artifactId>slider-agent</artifactId>
+      <version>${project.version}</version>
+      <scope>test</scope>
+      <type>tar.gz</type>
+    </dependency>
+
+    <dependency>
+      <groupId>com.beust</groupId>
+      <artifactId>jcommander</artifactId>
+    </dependency>
+
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-client</artifactId>
+    </dependency>
+ 
+  </dependencies>
+
+
+</project>


[16/50] [abbrv] git commit: SLIDER-149 revert to curator 2.4.1 . guava pain

Posted by sm...@apache.org.
SLIDER-149 revert to curator 2.4.1 . guava pain


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

Branch: refs/heads/master
Commit: 3a70ce9afb7d0c00e33a97af26f7029b7e55362b
Parents: a613e20
Author: Steve Loughran <st...@apache.org>
Authored: Mon Jul 7 16:56:22 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Jul 7 16:56:22 2014 +0100

----------------------------------------------------------------------
 pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3a70ce9a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c6980be..0076561 100644
--- a/pom.xml
+++ b/pom.xml
@@ -132,7 +132,7 @@
     <commons-compress.version>1.4.1</commons-compress.version>
     <commons-logging.version>1.1.3</commons-logging.version>
     <commons-io.version>2.4</commons-io.version>
-    <curator.version>2.5.0</curator.version>
+    <curator.version>2.4.1</curator.version>
     <easymock.version>3.1</easymock.version>
     <guava.version>11.0.2</guava.version>
     <gson.version>2.2.2</gson.version>


[05/50] [abbrv] git commit: SLIDER-200 double "mvn install" failing as rat rejects hbase/target/rat.txt

Posted by sm...@apache.org.
SLIDER-200 double "mvn install" failing as rat rejects hbase/target/rat.txt


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

Branch: refs/heads/master
Commit: 0151b9f53cdc46e39d80709c8090308f53bbfd3b
Parents: e9ddcf0
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 2 13:30:26 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 2 13:30:26 2014 +0100

----------------------------------------------------------------------
 pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0151b9f5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 460fb0d..0076561 100644
--- a/pom.xml
+++ b/pom.xml
@@ -300,6 +300,7 @@
             <exclude>**/httpfs-signature.secret</exclude>
             <exclude>**/dfs.exclude</exclude>
             <exclude>**/*.iml</exclude>
+            <exclude>**/rat.txt</exclude>
             <exclude>DISCLAIMER</exclude>
           </excludes>
         </configuration>


[17/50] [abbrv] git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-slider into develop

Posted by sm...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-slider into develop


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

Branch: refs/heads/master
Commit: bb1b86d89148329a03cf10de9c28b0da59d40110
Parents: 3a70ce9 8f93370
Author: Steve Loughran <st...@apache.org>
Authored: Mon Jul 7 16:56:32 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Jul 7 16:56:32 2014 +0100

----------------------------------------------------------------------
 app-packages/hbase/appConfig.json               |   1 +
 .../org/apache/slider/client/SliderClient.java  | 112 ++++++++++++++++++-
 .../slider/core/zk/BlockingZKWatcher.java       |   8 +-
 .../apache/slider/core/zk/ZKIntegration.java    |  23 +++-
 .../slider/providers/agent/AgentKeys.java       |   1 +
 .../providers/agent/AgentProviderService.java   |   1 +
 .../common/tools/TestZKIntegration.groovy       |  53 +++++++++
 7 files changed, 191 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[41/50] [abbrv] git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-slider into develop

Posted by sm...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-slider into develop


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

Branch: refs/heads/master
Commit: c011e7ac5a5120c87fa4800058af8f4cf8166636
Parents: 80424c1 859ca52
Author: tedyu <yu...@gmail.com>
Authored: Wed Jul 9 12:53:31 2014 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Wed Jul 9 12:53:31 2014 -0700

----------------------------------------------------------------------
 pom.xml                                         |  17 +-
 .../src/main/python/agent/AgentConfig.py        |   3 +
 .../src/main/python/agent/Controller.py         |  32 ++-
 slider-agent/src/main/python/agent/NetUtil.py   |   8 +-
 slider-agent/src/main/python/agent/main.py      |  15 +-
 slider-agent/src/main/python/agent/security.py  |   7 +-
 .../src/test/python/agent/TestController.py     |   4 +-
 slider-agent/src/test/python/agent/TestMain.py  |   8 +-
 .../src/test/python/agent/TestNetUtil.py        |   2 +-
 slider-core/pom.xml                             |   7 +
 .../java/org/apache/slider/api/StatusKeys.java  |   3 +
 .../org/apache/slider/common/SliderKeys.java    |  14 +
 .../apache/slider/common/tools/SliderUtils.java |   2 +-
 .../core/launch/ClasspathConstructor.java       |   1 -
 .../registry/info/CustomRegistryConstants.java  |   2 +-
 .../providers/AbstractProviderService.java      |   7 +-
 .../slider/providers/ProviderService.java       |   8 +-
 .../slider/providers/agent/AgentKeys.java       |   1 +
 .../providers/agent/AgentProviderService.java   |  16 +-
 .../slideram/SliderAMClientProvider.java        |   7 +-
 .../slideram/SliderAMProviderService.java       |  18 +-
 .../server/appmaster/SliderAppMaster.java       |  68 ++++-
 .../server/appmaster/web/AgentService.java      |  53 ++++
 .../server/appmaster/web/SliderAmIpFilter.java  |  18 +-
 .../slider/server/appmaster/web/WebAppApi.java  |   9 +-
 .../server/appmaster/web/WebAppApiImpl.java     |  16 +-
 .../appmaster/web/rest/AMWebServices.java       |   7 +-
 .../server/appmaster/web/rest/RestPaths.java    |   5 +-
 .../appmaster/web/rest/agent/AgentResource.java |  25 +-
 .../appmaster/web/rest/agent/AgentWebApp.java   | 203 +++++++++++++++
 .../web/rest/agent/AgentWebServices.java        |  40 +++
 .../services/security/CertificateManager.java   | 257 +++++++++++++++++++
 .../server/services/security/SecurityUtils.java | 209 +++++++++++++++
 .../services/security/SignCertResponse.java     |  67 +++++
 .../server/services/security/SignMessage.java   |  54 ++++
 .../providers/slideram/instance/appconf.json    |   5 +-
 .../main/resources/webapps/slideragent/.keep    |   0
 .../agent/TestAgentAMManagementWS.groovy        |  46 +++-
 .../model/mock/MockProviderService.groovy       |  12 +-
 .../view/TestClusterSpecificationBlock.groovy   |   2 +-
 .../web/view/TestContainerStatsBlock.groovy     |   2 +-
 .../appmaster/web/view/TestIndexBlock.groovy    |   2 +-
 .../agent/TestAgentProviderService.java         |   6 +-
 .../web/rest/agent/TestAMAgentWebServices.java  | 217 ++++++++--------
 .../management/TestAMManagementWebServices.java |   2 +-
 .../security/TestCertificateManager.java        |  60 +++++
 slider-core/src/test/python/agent/main.py       |   2 +
 .../providers/hbase/HBaseProviderService.java   |   5 +-
 48 files changed, 1381 insertions(+), 193 deletions(-)
----------------------------------------------------------------------



[43/50] [abbrv] git commit: remove duplicate ASF licenses in log4j.properties files

Posted by sm...@apache.org.
remove duplicate ASF licenses in log4j.properties files


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

Branch: refs/heads/master
Commit: b03c38f87b99cea960fb9fb826c595358ce60790
Parents: 66c677d
Author: Steve Loughran <st...@apache.org>
Authored: Thu Jul 10 11:16:03 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Jul 10 11:16:03 2014 +0100

----------------------------------------------------------------------
 src/test/clusters/c6401/slider/log4j.properties | 30 --------------------
 .../clusters/morzine/slider/log4j.properties    | 30 --------------------
 .../clusters/offline/slider/log4j.properties    | 30 --------------------
 .../clusters/remote/slider/log4j.properties     | 30 --------------------
 .../clusters/sandbox/slider/log4j.properties    | 30 --------------------
 .../ubuntu-secure/slider/log4j.properties       |  1 -
 6 files changed, 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b03c38f8/src/test/clusters/c6401/slider/log4j.properties
----------------------------------------------------------------------
diff --git a/src/test/clusters/c6401/slider/log4j.properties b/src/test/clusters/c6401/slider/log4j.properties
index 6211771..d814f14 100644
--- a/src/test/clusters/c6401/slider/log4j.properties
+++ b/src/test/clusters/c6401/slider/log4j.properties
@@ -11,36 +11,6 @@
 #   See the License for the specific language governing permissions and
 #   limitations under the License. See accompanying LICENSE file.
 #
-
-#
-# 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.
-#
-
-#   Licensed 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.
 # log4j configuration used during build and unit tests
 
 log4j.rootLogger=INFO,stdout

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b03c38f8/src/test/clusters/morzine/slider/log4j.properties
----------------------------------------------------------------------
diff --git a/src/test/clusters/morzine/slider/log4j.properties b/src/test/clusters/morzine/slider/log4j.properties
index 6211771..d814f14 100644
--- a/src/test/clusters/morzine/slider/log4j.properties
+++ b/src/test/clusters/morzine/slider/log4j.properties
@@ -11,36 +11,6 @@
 #   See the License for the specific language governing permissions and
 #   limitations under the License. See accompanying LICENSE file.
 #
-
-#
-# 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.
-#
-
-#   Licensed 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.
 # log4j configuration used during build and unit tests
 
 log4j.rootLogger=INFO,stdout

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b03c38f8/src/test/clusters/offline/slider/log4j.properties
----------------------------------------------------------------------
diff --git a/src/test/clusters/offline/slider/log4j.properties b/src/test/clusters/offline/slider/log4j.properties
index 6211771..d814f14 100644
--- a/src/test/clusters/offline/slider/log4j.properties
+++ b/src/test/clusters/offline/slider/log4j.properties
@@ -11,36 +11,6 @@
 #   See the License for the specific language governing permissions and
 #   limitations under the License. See accompanying LICENSE file.
 #
-
-#
-# 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.
-#
-
-#   Licensed 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.
 # log4j configuration used during build and unit tests
 
 log4j.rootLogger=INFO,stdout

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b03c38f8/src/test/clusters/remote/slider/log4j.properties
----------------------------------------------------------------------
diff --git a/src/test/clusters/remote/slider/log4j.properties b/src/test/clusters/remote/slider/log4j.properties
index 0f408f3..f672472 100644
--- a/src/test/clusters/remote/slider/log4j.properties
+++ b/src/test/clusters/remote/slider/log4j.properties
@@ -11,36 +11,6 @@
 #   See the License for the specific language governing permissions and
 #   limitations under the License. See accompanying LICENSE file.
 #
-
-#
-# 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.
-#
-
-#   Licensed 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.
 # log4j configuration used during build and unit tests
 
 log4j.rootLogger=INFO,stdout

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b03c38f8/src/test/clusters/sandbox/slider/log4j.properties
----------------------------------------------------------------------
diff --git a/src/test/clusters/sandbox/slider/log4j.properties b/src/test/clusters/sandbox/slider/log4j.properties
index 6211771..d814f14 100644
--- a/src/test/clusters/sandbox/slider/log4j.properties
+++ b/src/test/clusters/sandbox/slider/log4j.properties
@@ -11,36 +11,6 @@
 #   See the License for the specific language governing permissions and
 #   limitations under the License. See accompanying LICENSE file.
 #
-
-#
-# 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.
-#
-
-#   Licensed 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.
 # log4j configuration used during build and unit tests
 
 log4j.rootLogger=INFO,stdout

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/b03c38f8/src/test/clusters/ubuntu-secure/slider/log4j.properties
----------------------------------------------------------------------
diff --git a/src/test/clusters/ubuntu-secure/slider/log4j.properties b/src/test/clusters/ubuntu-secure/slider/log4j.properties
index 1bea885..73516f6 100644
--- a/src/test/clusters/ubuntu-secure/slider/log4j.properties
+++ b/src/test/clusters/ubuntu-secure/slider/log4j.properties
@@ -1,4 +1,3 @@
-
 #
 # Licensed to the Apache Software Foundation (ASF) under one
 #  or more contributor license agreements.  See the NOTICE file


[32/50] [abbrv] git commit: SLIDER-221 rpm is only built on the -Prpm profile

Posted by sm...@apache.org.
SLIDER-221 rpm is only built on the -Prpm profile


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

Branch: refs/heads/master
Commit: d44d4c1bf0e4a57d0cebb7408936d8f8a546442f
Parents: f62539b
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 9 14:17:48 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 9 14:17:48 2014 +0100

----------------------------------------------------------------------
 slider-assembly/pom.xml | 235 +++++++++++++++++++++++--------------------
 1 file changed, 124 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/d44d4c1b/slider-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/slider-assembly/pom.xml b/slider-assembly/pom.xml
index adfd875..95df363 100644
--- a/slider-assembly/pom.xml
+++ b/slider-assembly/pom.xml
@@ -189,117 +189,7 @@
         </executions>
       </plugin>
 
-      <!-- RPM -->
-
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>rpm-maven-plugin</artifactId>
-        <version>${maven-rpm-plugin.version}</version>
-        <executions>
-          <execution>
-            <id>generate-rpm</id>
-            <goals>
-              <goal>rpm</goal>
-            </goals>
-          </execution>
-        </executions>
-        <configuration>
-          <license>http://www.apache.org/licenses/LICENSE-2.0</license>
-          <name>slider</name>
-<!---->          <distribution></distribution>
-          <group>Applications/Engineering</group>
-          <!--<icon>src/main/resources/icon.gif</icon>-->
-          <!--<packager>ASF</packager>-->
-          <prefix>${rpm.basedir}</prefix>
-          <!--
-                    <changelogFile>src/changelog</changelogFile>
-          -->
-          <!--          <defineStatements>
-                      <defineStatement>_unpackaged_files_terminate_build 0</defineStatement>
-                    </defineStatements>-->
-          <provides>
-            <provide>apache-slider</provide>
-          </provides>
-          <mappings>
-            <mapping>
-              <directory>${rpm.basedir}</directory>
-              <!-- RW.R..R.. -->
-              <filemode>644</filemode>
-              
-              <username>${rpm.username}</username>
-              <groupname>${rpm.groupname}</groupname>
-              <sources>
-                <source>
-                  <location>target/classes</location>
-                </source>
-              </sources>
-            </mapping>
-            
-            <!-- binaries -->
-            <mapping>
-              <directory>${rpm.bindir}</directory>
-              <!-- RWXR.XR.X -->
-              <filemode>0755</filemode>
-              <username>${rpm.username}</username>
-              <groupname>${rpm.groupname}</groupname>
-              <sources>
-                <source>
-                  <location>src/main/scripts</location>
-                </source>
-              </sources>
-            </mapping>
-            
-            <!-- library -->
-            <mapping>
-              <directory>${rpm.libdir}</directory>
-              <!-- RW.R..R.. -->
-              <filemode>644</filemode>
-
-              <username>${rpm.username}</username>
-              <groupname>${rpm.groupname}</groupname>
-              <sources>
-                <source>
-                  <location>${src.libdir}</location>
-                </source>
-              </sources>
-            </mapping>
-                  
-          <!-- configuration -->
-            <mapping>
-              <directory>${rpm.confdir}</directory>
-              <configuration>true</configuration>
-              <filemode>0755</filemode>
-              <username>${rpm.username}</username>
-              <groupname>${rpm.groupname}</groupname>
-              <sources>
-                <source>
-                  <location>${src.confdir}</location>
-                </source>
-              </sources>
-            </mapping>
-
-          </mappings>
-<!--
-Scripts. 
--->
-          <preinstallScriptlet>
-            <script>echo "installing slider"</script>
-          </preinstallScriptlet>
-<!--
-          <postinstallScriptlet>
-            <scriptFile>src/main/scripts/postinstall</scriptFile>
-            <fileEncoding>utf-8</fileEncoding>
-          </postinstallScriptlet>
--->
-<!--
-          <preremoveScriptlet>
-            <scriptFile>src/main/scripts/preremove</scriptFile>
-            <fileEncoding>utf-8</fileEncoding>
-          </preremoveScriptlet>
--->
-        </configuration>
-      </plugin>
-
+ 
     </plugins>
     
     
@@ -313,6 +203,129 @@ Scripts.
     </plugins>
   </reporting>
 
+  <profiles>
+    <profile>
+      <id>rpm</id>
+      <build>
+        <plugins>
+          <!-- RPM -->
+
+          <plugin>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>rpm-maven-plugin</artifactId>
+            <version>${maven-rpm-plugin.version}</version>
+            <executions>
+              <execution>
+                <id>generate-rpm</id>
+                <goals>
+                  <goal>rpm</goal>
+                </goals>
+              </execution>
+            </executions>
+            <configuration>
+              <license>http://www.apache.org/licenses/LICENSE-2.0</license>
+              <name>slider</name>
+              <!---->
+              <distribution></distribution>
+              <group>Applications/Engineering</group>
+              <!--<icon>src/main/resources/icon.gif</icon>-->
+              <!--<packager>ASF</packager>-->
+              <prefix>${rpm.basedir}</prefix>
+              <!--
+                        <changelogFile>src/changelog</changelogFile>
+              -->
+              <!--          <defineStatements>
+                          <defineStatement>_unpackaged_files_terminate_build 0</defineStatement>
+                        </defineStatements>-->
+              <provides>
+                <provide>apache-slider</provide>
+              </provides>
+              <mappings>
+                <mapping>
+                  <directory>${rpm.basedir}</directory>
+                  <!-- RW.R..R.. -->
+                  <filemode>644</filemode>
+
+                  <username>${rpm.username}</username>
+                  <groupname>${rpm.groupname}</groupname>
+                  <sources>
+                    <source>
+                      <location>target/classes</location>
+                    </source>
+                  </sources>
+                </mapping>
+
+                <!-- binaries -->
+                <mapping>
+                  <directory>${rpm.bindir}</directory>
+                  <!-- RWXR.XR.X -->
+                  <filemode>0755</filemode>
+                  <username>${rpm.username}</username>
+                  <groupname>${rpm.groupname}</groupname>
+                  <sources>
+                    <source>
+                      <location>src/main/scripts</location>
+                    </source>
+                  </sources>
+                </mapping>
+
+                <!-- library -->
+                <mapping>
+                  <directory>${rpm.libdir}</directory>
+                  <!-- RW.R..R.. -->
+                  <filemode>644</filemode>
+
+                  <username>${rpm.username}</username>
+                  <groupname>${rpm.groupname}</groupname>
+                  <sources>
+                    <source>
+                      <location>${src.libdir}</location>
+                    </source>
+                  </sources>
+                </mapping>
+
+                <!-- configuration -->
+                <mapping>
+                  <directory>${rpm.confdir}</directory>
+                  <configuration>true</configuration>
+                  <filemode>0755</filemode>
+                  <username>${rpm.username}</username>
+                  <groupname>${rpm.groupname}</groupname>
+                  <sources>
+                    <source>
+                      <location>${src.confdir}</location>
+                    </source>
+                  </sources>
+                </mapping>
+
+              </mappings>
+              <!--
+              Scripts. 
+              -->
+              <preinstallScriptlet>
+                <script>echo "installing slider"</script>
+              </preinstallScriptlet>
+              <!--
+                        <postinstallScriptlet>
+                          <scriptFile>src/main/scripts/postinstall</scriptFile>
+                          <fileEncoding>utf-8</fileEncoding>
+                        </postinstallScriptlet>
+              -->
+              <!--
+                        <preremoveScriptlet>
+                          <scriptFile>src/main/scripts/preremove</scriptFile>
+                          <fileEncoding>utf-8</fileEncoding>
+                        </preremoveScriptlet>
+              -->
+            </configuration>
+          </plugin>
+
+        </plugins>
+      </build>
+    </profile>
+
+  </profiles>
+
   <dependencies>
 
     <dependency>


[22/50] [abbrv] git commit: SLIDER-134. Provide a default ZK node for apps

Posted by sm...@apache.org.
SLIDER-134. Provide a default ZK node for apps


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

Branch: refs/heads/master
Commit: 9be2bdd79b33b0561ed5ba87769b9a6471eb7473
Parents: f50ea0e
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Tue Jul 8 00:06:25 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Tue Jul 8 00:06:25 2014 -0700

----------------------------------------------------------------------
 app-packages/hbase/appConfig.json               |  4 +--
 .../org/apache/slider/client/SliderClient.java  | 17 +++++++-----
 slider-core/src/test/python/agent.py            | 17 ++++++------
 slider-core/src/test/python/agent/main.py       | 27 ++++++++++----------
 slider-core/src/test/python/echo.py             | 21 +++++++--------
 5 files changed, 47 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/9be2bdd7/app-packages/hbase/appConfig.json
----------------------------------------------------------------------
diff --git a/app-packages/hbase/appConfig.json b/app-packages/hbase/appConfig.json
index 3a810f7..07187ec 100644
--- a/app-packages/hbase/appConfig.json
+++ b/app-packages/hbase/appConfig.json
@@ -10,7 +10,7 @@
     "java_home": "/usr/jdk64/jdk1.7.0_45",
     "package_list": "files/hbase-${hbase.version}-bin.tar.gz",
     "site.global.app_user": "yarn",
-    "site.global.app_log_dir": "app/log",
+    "site.global.app_log_dir": "${AGENT_LOG_ROOT}/app/log",
     "site.global.app_pid_dir": "${AGENT_WORK_ROOT}/app/run",
     "site.global.app_root": "${AGENT_WORK_ROOT}/app/install/hbase-${hbase.version}",
     "site.global.app_install_dir": "${AGENT_WORK_ROOT}/app/install",
@@ -44,7 +44,7 @@
     "site.hbase-site.hbase.security.authentication": "simple",
     "site.hbase-site.hbase.defaults.for.version.skip": "true",
     "site.hbase-site.hbase.zookeeper.quorum": "${ZK_HOST}",
-    "site.hbase-site.zookeeper.znode.parent": "/hbase-unsecure",
+    "site.hbase-site.zookeeper.znode.parent": "${DEF_ZK_PATH}",
     "site.hbase-site.hbase.hstore.blockingStoreFiles": "10",
     "site.hbase-site.hbase.hregion.majorcompaction": "86400000",
     "site.hbase-site.hbase.security.authorization": "false",

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/9be2bdd7/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 286ebac..e762c1e 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
@@ -280,6 +280,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
   protected boolean deleteZookeeperNode(String clusterName) throws YarnException, IOException {
     String user = getUsername();
     String zkPath = ZKIntegration.mkClusterPath(user, clusterName);
+    Exception e = null;
     try {
       Configuration config = getConfig();
       if (!SliderUtils.isHadoopClusterSecure(config)) {
@@ -294,12 +295,16 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       } else {
         log.warn("Default zookeeper node is not available for secure cluster");
       }
-    } catch (InterruptedException e) {
-      log.warn("Unable to recursively delete zk node {}", zkPath, e);
-    } catch (KeeperException e) {
-      log.warn("Unable to recursively delete zk node {}", zkPath, e);
-    } catch (BadConfigException e) {
-      log.warn("Unable to recursively delete zk node {}", zkPath, e);
+    } catch (InterruptedException ignored) {
+      e = ignored;
+    } catch (KeeperException ignored) {
+      e = ignored;
+    } catch (BadConfigException ignored) {
+      e = ignored;
+    }
+    if (e != null) {
+      log.warn("Unable to recursively delete zk node {}", zkPath);
+      log.debug("Reason: ", e);
     }
 
     return false;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/9be2bdd7/slider-core/src/test/python/agent.py
----------------------------------------------------------------------
diff --git a/slider-core/src/test/python/agent.py b/slider-core/src/test/python/agent.py
index 4be2cd9..4177074 100644
--- a/slider-core/src/test/python/agent.py
+++ b/slider-core/src/test/python/agent.py
@@ -21,21 +21,22 @@ import logging
 import sys
 import datetime
 import time
-import argparse
+from optparse import OptionParser
 import os
 
 # A representative Agent code for the embedded agent
 def main():
   print "Executing echo"
-  print 'Argument List: {}'.format(str(sys.argv))
+  print 'Argument List: {0}'.format(str(sys.argv))
 
-  parser = argparse.ArgumentParser()
-  parser.add_argument('--log', dest='log_folder', help='log destination')
-  parser.add_argument('--config', dest='conf_folder', help='conf folder')
-  args = parser.parse_args()
-  if args.log_folder:
+  parser = OptionParser()
+  parser.add_option("--log", dest="log_folder", help="log destination")
+  parser.add_option("--config", dest="conf_folder", help="conf folder")
+  (options, args) = parser.parse_args()
+
+  if options.log_folder:
     log_file_name = "echo" + str(datetime.datetime.now()) + ".log"
-    log_file_path = os.path.join(args.log_folder, log_file_name)
+    log_file_path = os.path.join(options.log_folder, log_file_name)
     logging.basicConfig(filename=log_file_path, level=logging.DEBUG)
     print log_file_path
   logging.debug('Starting echo script ...')

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/9be2bdd7/slider-core/src/test/python/agent/main.py
----------------------------------------------------------------------
diff --git a/slider-core/src/test/python/agent/main.py b/slider-core/src/test/python/agent/main.py
index fd8b262..573b9a7 100755
--- a/slider-core/src/test/python/agent/main.py
+++ b/slider-core/src/test/python/agent/main.py
@@ -21,25 +21,26 @@ import logging
 import sys
 import datetime
 import time
-import argparse
+from optparse import OptionParser
 import os
 
 
 def main():
   print "Executing echo"
-  print 'Argument List: {}'.format(str(sys.argv))
-
-  parser = argparse.ArgumentParser()
-  parser.add_argument('--log', dest='log_folder', help='log destination')
-  parser.add_argument('--config', dest='conf_folder', help='conf folder')
-  parser.add_argument('--command', dest='command', help='command to execute')
-  parser.add_argument('--label', dest='label', help='label')
-  parser.add_argument('--host', dest='host', help='port')
-  parser.add_argument('--port', dest='port', help='host')
-  args = parser.parse_args()
-  if args.log_folder:
+  print 'Argument List: {0}'.format(str(sys.argv))
+
+  parser = OptionParser()
+  parser.add_option("--log", dest="log_folder", help="log destination")
+  parser.add_option("--config", dest="conf_folder", help="conf folder")
+  parser.add_option('--command', dest='command', help='command to execute')
+  parser.add_option('--label', dest='label', help='label')
+  parser.add_option('--host', dest='host', help='port')
+  parser.add_option('--port', dest='port', help='host')
+  (options, args) = parser.parse_args()
+
+  if options.log_folder:
     log_file_name = "echo" + str(datetime.datetime.now()) + ".log"
-    log_file_path = os.path.join(args.log_folder, log_file_name)
+    log_file_path = os.path.join(options.log_folder, log_file_name)
     logging.basicConfig(filename=log_file_path, level=logging.DEBUG)
     print log_file_path
   logging.debug('Starting echo script ...')

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/9be2bdd7/slider-core/src/test/python/echo.py
----------------------------------------------------------------------
diff --git a/slider-core/src/test/python/echo.py b/slider-core/src/test/python/echo.py
index 2bcab20..ea5e8ce 100644
--- a/slider-core/src/test/python/echo.py
+++ b/slider-core/src/test/python/echo.py
@@ -21,22 +21,23 @@ import logging
 import sys
 import datetime
 import time
-import argparse
+from optparse import OptionParser
 import os
 
 
 def main():
   print "Executing echo"
-  print 'Argument List: {}'.format(str(sys.argv))
-
-  parser = argparse.ArgumentParser()
-  parser.add_argument('--log', dest='log_folder', help='log destination')
-  parser.add_argument('--config', dest='conf_folder', help='conf folder')
-  parser.add_argument('--command', dest='command', help='command to execute')
-  args = parser.parse_args()
-  if args.log_folder:
+  print 'Argument List: {0}'.format(str(sys.argv))
+
+  parser = OptionParser()
+  parser.add_option("--log", dest="log_folder", help="log destination")
+  parser.add_option("--config", dest="conf_folder", help="conf folder")
+  parser.add_option('--command', dest='command', help='command to execute')
+  (options, args) = parser.parse_args()
+
+  if options.log_folder:
     log_file_name = "echo" + str(datetime.datetime.now()) + ".log"
-    log_file_path = os.path.join(args.log_folder, log_file_name)
+    log_file_path = os.path.join(options.log_folder, log_file_name)
     logging.basicConfig(filename=log_file_path, level=logging.DEBUG)
     print log_file_path
   logging.debug('Starting echo script ...')


[31/50] [abbrv] git commit: SLIDER-221 rpm in assembly package; specific config for HDP created; rpm currently runs on "package" target

Posted by sm...@apache.org.
SLIDER-221 rpm in assembly package; specific config for HDP created; rpm currently runs on "package" target


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

Branch: refs/heads/master
Commit: f62539bfee095944c1a5cc1b4527bd75a8fa233d
Parents: 497d11c
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 9 14:13:16 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 9 14:13:16 2014 +0100

----------------------------------------------------------------------
 slider-assembly/pom.xml                        | 123 +++++++++++++++++++
 slider-assembly/src/conf-hdp/log4j.properties  |  58 +++++++++
 slider-assembly/src/conf-hdp/slider-client.xml |  77 ++++++++++++
 slider-assembly/src/conf/slider-client.xml     |   3 -
 slider-install/pom.xml                         | 127 +-------------------
 5 files changed, 264 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f62539bf/slider-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/slider-assembly/pom.xml b/slider-assembly/pom.xml
index 9431fbc..adfd875 100644
--- a/slider-assembly/pom.xml
+++ b/slider-assembly/pom.xml
@@ -30,6 +30,18 @@
     <version>0.31.0-incubating-SNAPSHOT</version>
   </parent>
 
+
+  <properties>
+    <rpm.basedir>/usr/local/slider</rpm.basedir>
+    <rpm.confdir>${basedir}/conf</rpm.confdir>
+    <rpm.bindir>${basedir}/bin</rpm.bindir>
+    <rpm.libdir>${basedir}/lib</rpm.libdir>
+    <rpm.username>mapred</rpm.username>
+    <rpm.groupname>mapred</rpm.groupname>
+    <src.confdir>src/conf/hdp</src.confdir>
+    <src.libdir>${project.build.directory}/lib</src.libdir>
+  </properties>
+
   <build>
     <plugins>
       <!--read in a build.properties file if defined-->
@@ -177,6 +189,117 @@
         </executions>
       </plugin>
 
+      <!-- RPM -->
+
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>rpm-maven-plugin</artifactId>
+        <version>${maven-rpm-plugin.version}</version>
+        <executions>
+          <execution>
+            <id>generate-rpm</id>
+            <goals>
+              <goal>rpm</goal>
+            </goals>
+          </execution>
+        </executions>
+        <configuration>
+          <license>http://www.apache.org/licenses/LICENSE-2.0</license>
+          <name>slider</name>
+<!---->          <distribution></distribution>
+          <group>Applications/Engineering</group>
+          <!--<icon>src/main/resources/icon.gif</icon>-->
+          <!--<packager>ASF</packager>-->
+          <prefix>${rpm.basedir}</prefix>
+          <!--
+                    <changelogFile>src/changelog</changelogFile>
+          -->
+          <!--          <defineStatements>
+                      <defineStatement>_unpackaged_files_terminate_build 0</defineStatement>
+                    </defineStatements>-->
+          <provides>
+            <provide>apache-slider</provide>
+          </provides>
+          <mappings>
+            <mapping>
+              <directory>${rpm.basedir}</directory>
+              <!-- RW.R..R.. -->
+              <filemode>644</filemode>
+              
+              <username>${rpm.username}</username>
+              <groupname>${rpm.groupname}</groupname>
+              <sources>
+                <source>
+                  <location>target/classes</location>
+                </source>
+              </sources>
+            </mapping>
+            
+            <!-- binaries -->
+            <mapping>
+              <directory>${rpm.bindir}</directory>
+              <!-- RWXR.XR.X -->
+              <filemode>0755</filemode>
+              <username>${rpm.username}</username>
+              <groupname>${rpm.groupname}</groupname>
+              <sources>
+                <source>
+                  <location>src/main/scripts</location>
+                </source>
+              </sources>
+            </mapping>
+            
+            <!-- library -->
+            <mapping>
+              <directory>${rpm.libdir}</directory>
+              <!-- RW.R..R.. -->
+              <filemode>644</filemode>
+
+              <username>${rpm.username}</username>
+              <groupname>${rpm.groupname}</groupname>
+              <sources>
+                <source>
+                  <location>${src.libdir}</location>
+                </source>
+              </sources>
+            </mapping>
+                  
+          <!-- configuration -->
+            <mapping>
+              <directory>${rpm.confdir}</directory>
+              <configuration>true</configuration>
+              <filemode>0755</filemode>
+              <username>${rpm.username}</username>
+              <groupname>${rpm.groupname}</groupname>
+              <sources>
+                <source>
+                  <location>${src.confdir}</location>
+                </source>
+              </sources>
+            </mapping>
+
+          </mappings>
+<!--
+Scripts. 
+-->
+          <preinstallScriptlet>
+            <script>echo "installing slider"</script>
+          </preinstallScriptlet>
+<!--
+          <postinstallScriptlet>
+            <scriptFile>src/main/scripts/postinstall</scriptFile>
+            <fileEncoding>utf-8</fileEncoding>
+          </postinstallScriptlet>
+-->
+<!--
+          <preremoveScriptlet>
+            <scriptFile>src/main/scripts/preremove</scriptFile>
+            <fileEncoding>utf-8</fileEncoding>
+          </preremoveScriptlet>
+-->
+        </configuration>
+      </plugin>
+
     </plugins>
     
     

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f62539bf/slider-assembly/src/conf-hdp/log4j.properties
----------------------------------------------------------------------
diff --git a/slider-assembly/src/conf-hdp/log4j.properties b/slider-assembly/src/conf-hdp/log4j.properties
new file mode 100644
index 0000000..3c0d08c
--- /dev/null
+++ b/slider-assembly/src/conf-hdp/log4j.properties
@@ -0,0 +1,58 @@
+# 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.
+#
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=INFO,stdout
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+
+# log layout skips stack-trace creation operations by avoiding line numbers and method
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} - %m%n
+
+# debug edition is much more expensive
+#log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n
+
+
+log4j.appender.subprocess=org.apache.log4j.ConsoleAppender
+log4j.appender.subprocess.layout=org.apache.log4j.PatternLayout
+log4j.appender.subprocess.layout.ConversionPattern=[%c{1}]: %m%n
+#log4j.logger.org.apache.slider.yarn.appmaster.SliderAppMasterer.master=INFO,subprocess
+
+# for debugging Slider
+#log4j.logger.org.apache.slider=DEBUG
+#log4j.logger.org.apache.slider=DEBUG
+
+# uncomment to debug service lifecycle issues
+#log4j.logger.org.apache.hadoop.yarn.service.launcher=DEBUG
+#log4j.logger.org.apache.hadoop.yarn.service=DEBUG
+
+# uncomment for YARN operations
+#log4j.logger.org.apache.hadoop.yarn.client=DEBUG
+
+# uncomment this to debug security problems
+#log4j.logger.org.apache.hadoop.security=DEBUG
+
+#crank back on some noise
+log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR
+log4j.logger.org.apache.hadoop.hdfs=WARN
+
+
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor=WARN
+log4j.logger.org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl=WARN
+log4j.logger.org.apache.zookeeper=WARN

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f62539bf/slider-assembly/src/conf-hdp/slider-client.xml
----------------------------------------------------------------------
diff --git a/slider-assembly/src/conf-hdp/slider-client.xml b/slider-assembly/src/conf-hdp/slider-client.xml
new file mode 100644
index 0000000..f844106
--- /dev/null
+++ b/slider-assembly/src/conf-hdp/slider-client.xml
@@ -0,0 +1,77 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+   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.
+-->
+
+<!--
+  Properties set here are picked up in the client.
+  They are not passed to the AM -though the filesystem
+  binding details (URL And principal) are added to the
+  hbase-site.xml file when a cluster is created.
+-->
+<configuration>
+
+  <property>
+    <name>yarn.log-aggregation-enable</name>
+    <value>true</value>
+  </property>
+  
+
+  <property>
+    <name>slider.yarn.queue</name>
+    <value>default</value>
+    <description>YARN queue for the Application Master</description>
+  </property>
+
+
+  <property>
+    <name>yarn.application.classpath</name>
+    <value>
+      /etc/hadoop/conf,/usr/lib/hadoop/*,/usr/lib/hadoop/lib/*,/usr/lib/hadoop-hdfs/*,/usr/lib/hadoop-hdfs/lib/*,/usr/lib/hadoop-yarn/*,/usr/lib/hadoop-yarn/lib/*,/usr/lib/hadoop-mapreduce/*,/usr/lib/hadoop-mapreduce/lib/*
+    </value>
+  </property>
+  
+<!--
+
+  <property>
+    <name>yarn.resourcemanager.address</name>
+    <value>master:8032</value>
+  </property>
+
+  <property>
+    <name>fs.defaultFS</name>
+    <value>hdfs://master:9090</value>
+  </property>
+
+  <property>
+    <name>yarn.resourcemanager.principal</name>
+    <value>yarn/master@MINICLUSTER</value>
+  </property>
+
+  <property>
+    <name>slider.security.enabled</name>
+    <value>true</value>
+  </property>
+
+  <property>
+    <name>dfs.namenode.kerberos.principal</name>
+    <value>hdfs/master@MINICLUSTER</value>
+  </property>
+-->
+
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f62539bf/slider-assembly/src/conf/slider-client.xml
----------------------------------------------------------------------
diff --git a/slider-assembly/src/conf/slider-client.xml b/slider-assembly/src/conf/slider-client.xml
index d0c8f69..bd17254 100644
--- a/slider-assembly/src/conf/slider-client.xml
+++ b/slider-assembly/src/conf/slider-client.xml
@@ -19,9 +19,6 @@
 
 <!--
   Properties set here are picked up in the client.
-  They are not passed to the AM -though the filesystem
-  binding details (URL And principal) are added to the
-  hbase-site.xml file when a cluster is created.
 -->
 <configuration>
   <property>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f62539bf/slider-install/pom.xml
----------------------------------------------------------------------
diff --git a/slider-install/pom.xml b/slider-install/pom.xml
index a00dca9..df061a7 100644
--- a/slider-install/pom.xml
+++ b/slider-install/pom.xml
@@ -31,6 +31,12 @@
     <version>0.31.0-incubating-SNAPSHOT</version>
   </parent>
 
+  <properties>
+    <basedir>/usr/local/slider</basedir>
+    <confdir>${basedir}/conf</confdir>
+    <bindir>${basedir}/bin</bindir>
+  </properties>
+  
   <build>
     <plugins>
       <!--read in a build.properties file if defined-->
@@ -55,127 +61,6 @@
         </executions>
       </plugin>
 
-
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>rpm-maven-plugin</artifactId>
-        <version>${maven-rpm-plugin.version}</version>
-        <executions>
-          <execution>
-            <id>generate-rpm</id>
-            <goals>
-              <goal>rpm</goal>
-            </goals>
-          </execution>
-        </executions>
-        <configuration>
-          <license>http://www.apache.org/licenses/LICENSE-2.0</license>
-          <distribution>Trash 2005</distribution>
-          <group>Application/Collectors</group>
-          <icon>src/main/resources/icon.gif</icon>
-          <packager>SWWDC</packager>
-          <prefix>/usr/local</prefix>
-          <changelogFile>src/changelog</changelogFile>
-          <defineStatements>
-            <defineStatement>_unpackaged_files_terminate_build 0</defineStatement>
-          </defineStatements>
-          <mappings>
-            <mapping>
-              <directory>/usr/local/bin/landfill</directory>
-              <filemode>440</filemode>
-              <username>dumper</username>
-              <groupname>dumpgroup</groupname>
-              <sources>
-                <source>
-                  <location>target/classes</location>
-                </source>
-              </sources>
-            </mapping>
-            <mapping>
-              <directory>/usr/local/doc/landfill</directory>
-              <documentation>true</documentation>
-              <filemode>444</filemode>
-              <username>dumper</username>
-              <groupname>dumpgroup</groupname>
-              <sources>
-                <source>
-                  <location>target/site</location>
-                </source>
-              </sources>
-            </mapping>
-            <mapping>
-              <directory>/usr/local/lib</directory>
-              <filemode>750</filemode>
-              <username>dumper</username>
-              <groupname>dumpgroup</groupname>
-              <dependency>
-                <includes>
-                  <include>jmock:jmock</include>
-                  <include>javax.servlet:servlet-api:2.4</include>
-                </includes>
-                <excludes>
-                  <exclude>junit:junit</exclude>
-                </excludes>
-              </dependency>
-            </mapping>
-            <mapping>
-              <directory>/usr/local/bin</directory>
-              <filemode>750</filemode>
-              <username>dumper</username>
-              <groupname>dumpgroup</groupname>
-              <sources>
-                <source>
-                  <location>src/main/bin</location>
-                </source>
-                <softlinkSource>
-                  <location>/usr/local/bin/new.sh</location>
-                  <destination>old.sh</destination>
-                </softlinkSource>
-              </sources>
-            </mapping>
-            <mapping>
-              <directory>/usr/local/oldbin</directory>
-              <filemode>750</filemode>
-              <username>dumper</username>
-              <groupname>dumpgroup</groupname>
-              <sources>
-                <softlinkSource>
-                  <location>/usr/local/bin</location>
-                </softlinkSource>
-              </sources>
-            </mapping>
-            <mapping>
-              <directory>/usr/local/conf/landfill</directory>
-              <configuration>true</configuration>
-              <filemode>640</filemode>
-              <username>dumper</username>
-              <groupname>dumpgroup</groupname>
-              <sources>
-                <source>
-                  <location>src/main/conf</location>
-                </source>
-              </sources>
-            </mapping>
-            <mapping>
-              <directory>/usr/local/log/landfill</directory>
-              <filemode>750</filemode>
-              <username>dumper</username>
-              <groupname>dumpgroup</groupname>
-            </mapping>
-          </mappings>
-          <preinstallScriptlet>
-            <script>echo "installing now"</script>
-          </preinstallScriptlet>
-          <postinstallScriptlet>
-            <scriptFile>src/main/scripts/postinstall</scriptFile>
-            <fileEncoding>utf-8</fileEncoding>
-          </postinstallScriptlet>
-          <preremoveScriptlet>
-            <scriptFile>src/main/scripts/preremove</scriptFile>
-            <fileEncoding>utf-8</fileEncoding>
-          </preremoveScriptlet>
-        </configuration>
-      </plugin>
     </plugins>
     
     


[36/50] [abbrv] git commit: SLIDER-122 pom indentation

Posted by sm...@apache.org.
SLIDER-122 pom indentation


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

Branch: refs/heads/master
Commit: c1a53057b779c276186918b5e95a9a7ab8705e7f
Parents: 729b562
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 9 18:34:41 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 9 18:34:41 2014 +0100

----------------------------------------------------------------------
 slider-assembly/pom.xml | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/c1a53057/slider-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/slider-assembly/pom.xml b/slider-assembly/pom.xml
index 9fff10c..2a312bb 100644
--- a/slider-assembly/pom.xml
+++ b/slider-assembly/pom.xml
@@ -37,7 +37,7 @@
     <rpm.groupname>hadoop</rpm.groupname>
     <src.confdir>src/conf-hdp</src.confdir>
     <src.libdir>${project.build.directory}/lib</src.libdir>
-   <src.agent.ini.dir>${project.build.directory}/../../slider-agent/conf</src.agent.ini.dir>
+    <src.agent.ini.dir>${project.build.directory}/../../slider-agent/conf</src.agent.ini.dir>
   </properties>
 
   <build>
@@ -325,10 +325,10 @@
                   <sources>
                     <source>
                       <location>${src.agent.ini.dir}</location>
-<includes>
-<include>*.ini
-</include>
-</includes>
+                      <includes>
+                         <include>*.ini
+                        </include>
+                      </includes>
                     </source>
                   </sources>
                 </mapping>


[44/50] [abbrv] git commit: SLIDER-225. mvn clean install should succeed without any additional parameter

Posted by sm...@apache.org.
SLIDER-225. mvn clean install should succeed without any additional parameter


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

Branch: refs/heads/master
Commit: ab71b6546d6cfaba5a278df56133a983fb4cc04f
Parents: b03c38f
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Thu Jul 10 12:36:24 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Thu Jul 10 12:36:24 2014 -0700

----------------------------------------------------------------------
 slider-providers/hbase/hbase-funtests/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/ab71b654/slider-providers/hbase/hbase-funtests/pom.xml
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/hbase-funtests/pom.xml b/slider-providers/hbase/hbase-funtests/pom.xml
index 62db32e..cfebfc2 100644
--- a/slider-providers/hbase/hbase-funtests/pom.xml
+++ b/slider-providers/hbase/hbase-funtests/pom.xml
@@ -107,7 +107,7 @@
             <java.security.krb5.realm>${slider.test.java.security.krb5.realm}</java.security.krb5.realm>
             <java.security.krb5.kdc>${slider.test.java.security.krb5.kdc}</java.security.krb5.kdc>
             <!-- this property must be supplied-->
-            <slider.conf.dir>${slider.conf.dir}</slider.conf.dir>
+            <slider.conf.dir>../../../src/test/clusters/offline/slider</slider.conf.dir>
             <slider.bin.dir>../../../slider-assembly/target/slider-${project.version}-all/slider-${project.version}</slider.bin.dir>
           </systemPropertyVariables>
           <includes>


[07/50] [abbrv] git commit: SLIDER-201 jenkins windows support - fail fast wasnt static

Posted by sm...@apache.org.
SLIDER-201 jenkins windows support - fail fast wasnt static


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

Branch: refs/heads/master
Commit: 6d96ba95659c2f47771eebbc1c632c89c3ee173f
Parents: a6cbab1
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 2 16:37:31 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 2 16:37:31 2014 +0100

----------------------------------------------------------------------
 .../groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6d96ba95/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy b/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
index a5c8710..80a83a4 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
@@ -130,9 +130,9 @@ public abstract class YarnMiniClusterTestBase extends ServiceLauncherBaseTest {
           DEFAULT_TEST_TIMEOUT_SECONDS * 1000)
   )
   @BeforeClass
-  public void checkWindowsSupport() {
+  public static void checkWindowsSupport() {
     if (Shell.WINDOWS) {
-      Assert.assertNotNull("winutils.exe not found", Shell.WINUTILS)
+      assertNotNull("winutils.exe not found", Shell.WINUTILS)
     }
   } 
 


[33/50] [abbrv] git commit: SLIDER-221 RPM is building; contents manually verified

Posted by sm...@apache.org.
SLIDER-221 RPM is building; contents manually verified


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

Branch: refs/heads/master
Commit: 4e0265487073f22b4f37b57e183891561ed7e21e
Parents: d44d4c1
Author: Steve Loughran <st...@hortonworks.com>
Authored: Wed Jul 9 15:32:07 2014 +0100
Committer: Steve Loughran <st...@hortonworks.com>
Committed: Wed Jul 9 15:32:07 2014 +0100

----------------------------------------------------------------------
 slider-assembly/pom.xml | 12 +++++-------
 1 file changed, 5 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/4e026548/slider-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/slider-assembly/pom.xml b/slider-assembly/pom.xml
index 95df363..8bc8f12 100644
--- a/slider-assembly/pom.xml
+++ b/slider-assembly/pom.xml
@@ -33,12 +33,12 @@
 
   <properties>
     <rpm.basedir>/usr/local/slider</rpm.basedir>
-    <rpm.confdir>${basedir}/conf</rpm.confdir>
-    <rpm.bindir>${basedir}/bin</rpm.bindir>
-    <rpm.libdir>${basedir}/lib</rpm.libdir>
+    <rpm.confdir>${rpm.basedir}/conf</rpm.confdir>
+    <rpm.bindir>${rpm.basedir}/bin</rpm.bindir>
+    <rpm.libdir>${rpm.basedir}/lib</rpm.libdir>
     <rpm.username>mapred</rpm.username>
     <rpm.groupname>mapred</rpm.groupname>
-    <src.confdir>src/conf/hdp</src.confdir>
+    <src.confdir>src/conf-hdp</src.confdir>
     <src.libdir>${project.build.directory}/lib</src.libdir>
   </properties>
 
@@ -249,9 +249,7 @@
                   <username>${rpm.username}</username>
                   <groupname>${rpm.groupname}</groupname>
                   <sources>
-                    <source>
-                      <location>target/classes</location>
-                    </source>
+
                   </sources>
                 </mapping>
 


[02/50] [abbrv] git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-slider into develop

Posted by sm...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-slider into develop


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

Branch: refs/heads/master
Commit: e4723f5e538d15c39320c3651d6bf4c400c59400
Parents: 613dbec be94df3
Author: Steve Loughran <st...@apache.org>
Authored: Tue Jul 1 21:44:44 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Tue Jul 1 21:44:44 2014 +0100

----------------------------------------------------------------------
 app-packages/command-logger/README.txt          |  22 ++++
 .../command-logger/application-pkg/README.txt   |  24 ++++
 .../command-logger/application-pkg/pom.xml      |  83 +++++++++++++
 .../src/command_logger/README.txt               |  18 +++
 .../src/command_logger/operations.log           |  24 ++++
 .../src/packages/tarball/all.xml                |  35 ++++++
 .../command-logger/slider-pkg/appConfig.json    |  27 ++++
 .../slider-pkg/configuration/cl-site.xml        |  34 ++++++
 .../command-logger/slider-pkg/metainfo.xml      |  52 ++++++++
 .../slider-pkg/package/scripts/cl.py            |  89 ++++++++++++++
 .../slider-pkg/package/scripts/params.py        |  31 +++++
 .../package/templates/operations.log.j2         |  22 ++++
 app-packages/command-logger/slider-pkg/pom.xml  | 122 +++++++++++++++++++
 .../command-logger/slider-pkg/resources.json    |  15 +++
 .../slider-pkg/src/assembly/command-logger.xml  |  78 ++++++++++++
 app-packages/hbase/README.txt                   |   2 +-
 app-packages/hbase/appConfig.json               |   4 +-
 pom.xml                                         |   2 +
 .../test_command_log/appConfig.json             |   6 +-
 .../test_command_log/appConfig_fast_no_reg.json |   6 +-
 .../test_command_log/appConfig_no_hb.json       |   6 +-
 .../test_command_log/configuration/cl-site.xml  |  34 ------
 .../app_packages/test_command_log/metainfo.xml  |  52 --------
 .../package/files/command_log_10.tar            | Bin 2560 -> 0 bytes
 .../test_command_log/package/scripts/cl.py      |  89 --------------
 .../test_command_log/package/scripts/params.py  |  31 -----
 .../package/templates/operations.log.j2         |  22 ----
 slider-funtest/pom.xml                          |  52 +++++++-
 .../lifecycle/AgentCommandTestBase.groovy       |  21 +---
 29 files changed, 747 insertions(+), 256 deletions(-)
----------------------------------------------------------------------



[34/50] [abbrv] git commit: SLIDER-221 tuning RPM install after testing

Posted by sm...@apache.org.
SLIDER-221 tuning RPM install after testing


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

Branch: refs/heads/master
Commit: 27fe84d02aee5bcabf9c69240f5ced70d95ae14f
Parents: 4e02654
Author: Steve Loughran <st...@apache.org>
Authored: Wed Jul 9 16:40:55 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Wed Jul 9 16:40:55 2014 +0100

----------------------------------------------------------------------
 slider-assembly/pom.xml | 55 +++++++++++++++++++++++++++++++++++++-------
 1 file changed, 47 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/27fe84d0/slider-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/slider-assembly/pom.xml b/slider-assembly/pom.xml
index 8bc8f12..9fff10c 100644
--- a/slider-assembly/pom.xml
+++ b/slider-assembly/pom.xml
@@ -19,11 +19,7 @@
   <artifactId>slider-assembly</artifactId>
   <name>Slider Assembly</name>
   <packaging>pom</packaging>
-  <description>
-    
-    This is the module that does the packaging and shell entry
-    points for Slider
-  </description>
+  <description>Apache Slider is a tool for dynamically deploying distributed applications to an Apache YARN cluster</description>
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
@@ -32,14 +28,16 @@
 
 
   <properties>
-    <rpm.basedir>/usr/local/slider</rpm.basedir>
+    <rpm.basedir>/usr/lib/slider</rpm.basedir>
     <rpm.confdir>${rpm.basedir}/conf</rpm.confdir>
     <rpm.bindir>${rpm.basedir}/bin</rpm.bindir>
     <rpm.libdir>${rpm.basedir}/lib</rpm.libdir>
+    <rpm.agentdir>${rpm.basedir}/agent</rpm.agentdir>
     <rpm.username>mapred</rpm.username>
-    <rpm.groupname>mapred</rpm.groupname>
+    <rpm.groupname>hadoop</rpm.groupname>
     <src.confdir>src/conf-hdp</src.confdir>
     <src.libdir>${project.build.directory}/lib</src.libdir>
+   <src.agent.ini.dir>${project.build.directory}/../../slider-agent/conf</src.agent.ini.dir>
   </properties>
 
   <build>
@@ -225,6 +223,10 @@
             <configuration>
               <license>http://www.apache.org/licenses/LICENSE-2.0</license>
               <name>slider</name>
+              <version>${project.version}</version>
+<!--
+              <release>${project.version}</release>
+-->
               <!---->
               <distribution></distribution>
               <group>Applications/Engineering</group>
@@ -296,13 +298,50 @@
                   </sources>
                 </mapping>
 
+                <!-- agent -->
+                <mapping>
+                  <directory>${rpm.agentdir}</directory>
+                  <configuration>true</configuration>
+                  <filemode>0755</filemode>
+                  <username>${rpm.username}</username>
+                  <groupname>${rpm.groupname}</groupname>
+                  <sources>
+                    <source>
+                      <location>${project.build.directory}/agent</location>
+      <includes>
+        <include>slider-agent.tar.gz</include>
+      </includes>
+                    </source>
+                  </sources>
+                </mapping>
+
+                <!-- agent.ini -->
+                <mapping>
+                  <directory>${rpm.agentdir}/conf</directory>
+                  <configuration>true</configuration>
+                  <filemode>0755</filemode>
+                  <username>${rpm.username}</username>
+                  <groupname>${rpm.groupname}</groupname>
+                  <sources>
+                    <source>
+                      <location>${src.agent.ini.dir}</location>
+<includes>
+<include>*.ini
+</include>
+</includes>
+                    </source>
+                  </sources>
+                </mapping>
               </mappings>
               <!--
-              Scripts. 
+              Scripts. Very dangerous in RPMs unless you know exactly what you are doing.
+              It's very easy to break the uninstall process, in particular.
               -->
+<!--
               <preinstallScriptlet>
                 <script>echo "installing slider"</script>
               </preinstallScriptlet>
+-->
               <!--
                         <postinstallScriptlet>
                           <scriptFile>src/main/scripts/postinstall</scriptFile>


[18/50] [abbrv] git commit: SLIDER-212 Add hbase dependencies for slider-providers/hbase/hbase-funtests

Posted by sm...@apache.org.
SLIDER-212 Add hbase dependencies for slider-providers/hbase/hbase-funtests


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

Branch: refs/heads/master
Commit: 0d0ebe9d8bd50e30d7b866389c480bb33133acc4
Parents: bb1b86d
Author: tedyu <yu...@gmail.com>
Authored: Mon Jul 7 10:12:29 2014 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Jul 7 10:12:29 2014 -0700

----------------------------------------------------------------------
 pom.xml                                              |  6 ++++++
 slider-providers/hbase/slider-hbase-provider/pom.xml | 10 ++++++++++
 2 files changed, 16 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d0ebe9d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0076561..6c86b33 100644
--- a/pom.xml
+++ b/pom.xml
@@ -635,6 +635,12 @@
 
       <dependency>
         <groupId>org.apache.hbase</groupId>
+        <artifactId>hbase-protocol</artifactId>
+        <version>${hbase.version}</version>
+      </dependency>
+
+      <dependency>
+        <groupId>org.apache.hbase</groupId>
         <artifactId>hbase-it</artifactId>
         <version>${hbase.version}</version>
         <classifier>tests</classifier>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0d0ebe9d/slider-providers/hbase/slider-hbase-provider/pom.xml
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/slider-hbase-provider/pom.xml b/slider-providers/hbase/slider-hbase-provider/pom.xml
index c8e17ce..1cd5e66 100644
--- a/slider-providers/hbase/slider-hbase-provider/pom.xml
+++ b/slider-providers/hbase/slider-hbase-provider/pom.xml
@@ -188,6 +188,16 @@
 
     <dependency>
       <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-protocol</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-common</artifactId>
       <classifier>tests</classifier>
       <scope>test</scope>


[48/50] [abbrv] git commit: Merge branch 'release/slider-0.40'

Posted by sm...@apache.org.
Merge branch 'release/slider-0.40'


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

Branch: refs/heads/master
Commit: ddf7338759480fd29bae5fb72d1105c0a3e8647d
Parents: d595362 e09a6a8
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Thu Jul 10 21:14:14 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Thu Jul 10 21:14:14 2014 -0700

----------------------------------------------------------------------
 .gitignore                                      |    10 +-
 LICENSE                                         |   423 +
 LICENSE.txt                                     |   423 -
 NOTICE                                          |     5 +
 NOTICE.txt                                      |     5 -
 app-packages/accumulo-v1_5/README.txt           |    33 -
 app-packages/accumulo-v1_5/appConfig.json       |    61 -
 .../configuration/accumulo-site.xml             |   111 -
 .../accumulo-v1_5/configuration/global.xml      |    94 -
 app-packages/accumulo-v1_5/jmx_metrics.json     |    41 -
 app-packages/accumulo-v1_5/metainfo.xml         |   145 -
 .../files/accumulo-1.5.1-bin.tar.gz.REPLACE     |    14 -
 .../package/files/accumulo-metrics.xml          |    60 -
 .../accumulo-v1_5/package/files/auditLog.xml    |    41 -
 app-packages/accumulo-v1_5/package/files/gc     |    16 -
 .../package/files/generic_logger.xml            |    83 -
 .../package/files/log4j.properties              |    41 -
 .../accumulo-v1_5/package/files/masters         |    16 -
 .../accumulo-v1_5/package/files/monitor         |    16 -
 .../package/files/monitor_logger.xml            |    64 -
 app-packages/accumulo-v1_5/package/files/slaves |    16 -
 .../accumulo-v1_5/package/files/tracers         |    16 -
 .../accumulo-v1_5/package/scripts/__init__.py   |    19 -
 .../package/scripts/accumulo_client.py          |    43 -
 .../package/scripts/accumulo_configuration.py   |   126 -
 .../package/scripts/accumulo_gc.py              |    24 -
 .../package/scripts/accumulo_master.py          |    24 -
 .../package/scripts/accumulo_monitor.py         |    24 -
 .../package/scripts/accumulo_script.py          |    68 -
 .../package/scripts/accumulo_service.py         |    52 -
 .../package/scripts/accumulo_tracer.py          |    24 -
 .../package/scripts/accumulo_tserver.py         |    24 -
 .../accumulo-v1_5/package/scripts/params.py     |    62 -
 .../package/scripts/status_params.py            |    26 -
 .../package/templates/accumulo-env.sh.j2        |    42 -
 app-packages/accumulo-v1_5/resources.json       |    31 -
 app-packages/accumulo/LICENSE.txt               |   261 +
 app-packages/accumulo/NOTICE.txt                |     8 +
 app-packages/accumulo/README.txt                |    47 +
 app-packages/accumulo/appConfig.json            |    61 +
 .../accumulo/configuration/accumulo-site.xml    |   111 +
 app-packages/accumulo/configuration/global.xml  |    94 +
 app-packages/accumulo/jmx_metrics.json          |    41 +
 app-packages/accumulo/metainfo.xml              |   147 +
 .../accumulo/package/files/accumulo-metrics.xml |    60 +
 .../accumulo/package/files/auditLog.xml         |    41 +
 app-packages/accumulo/package/files/gc          |    16 +
 .../accumulo/package/files/generic_logger.xml   |    83 +
 .../accumulo/package/files/log4j.properties     |    41 +
 app-packages/accumulo/package/files/masters     |    16 +
 app-packages/accumulo/package/files/monitor     |    16 +
 .../accumulo/package/files/monitor_logger.xml   |    64 +
 app-packages/accumulo/package/files/slaves      |    16 +
 app-packages/accumulo/package/files/tracers     |    16 +
 .../accumulo/package/scripts/__init__.py        |    19 +
 .../accumulo/package/scripts/accumulo_client.py |    43 +
 .../package/scripts/accumulo_configuration.py   |   135 +
 .../accumulo/package/scripts/accumulo_gc.py     |    24 +
 .../accumulo/package/scripts/accumulo_master.py |    24 +
 .../package/scripts/accumulo_monitor.py         |    24 +
 .../accumulo/package/scripts/accumulo_script.py |   110 +
 .../package/scripts/accumulo_service.py         |    52 +
 .../accumulo/package/scripts/accumulo_tracer.py |    24 +
 .../package/scripts/accumulo_tserver.py         |    24 +
 app-packages/accumulo/package/scripts/params.py |    76 +
 .../accumulo/package/scripts/status_params.py   |    26 +
 .../package/templates/accumulo-env.sh.j2        |    42 +
 app-packages/accumulo/pom.xml                   |   198 +
 app-packages/accumulo/resources.json            |    36 +
 app-packages/accumulo/src/assembly/accumulo.xml |    72 +
 .../AccumuloAgentCommandTestBase.groovy         |    52 +
 .../funtest/accumulo/AccumuloBasicIT.groovy     |   117 +
 .../accumulo/AccumuloMonitorSSLIT.groovy        |    73 +
 .../funtest/accumulo/AccumuloReadWriteIT.groovy |   116 +
 .../StubToForceGroovyTestsToCompile.java        |    22 +
 .../test/resources/appConfig_monitor_ssl.json   |    62 +
 .../accumulo/src/test/resources/resources.json  |    36 +
 app-packages/command-logger/README.txt          |    22 +
 .../command-logger/application-pkg/README.txt   |    24 +
 .../command-logger/application-pkg/pom.xml      |    83 +
 .../src/command_logger/README.txt               |    18 +
 .../src/command_logger/operations.log           |    24 +
 .../src/packages/tarball/all.xml                |    35 +
 .../command-logger/slider-pkg/appConfig.json    |    26 +
 .../slider-pkg/configuration/cl-site.xml        |    34 +
 .../command-logger/slider-pkg/metainfo.xml      |    52 +
 .../slider-pkg/package/scripts/cl.py            |    89 +
 .../slider-pkg/package/scripts/params.py        |    31 +
 .../package/templates/operations.log.j2         |    22 +
 app-packages/command-logger/slider-pkg/pom.xml  |   122 +
 .../command-logger/slider-pkg/resources.json    |    15 +
 .../slider-pkg/src/assembly/command-logger.xml  |    78 +
 app-packages/hbase-v0_96/README.txt             |    33 -
 app-packages/hbase-v0_96/appConfig.json         |    67 -
 .../hbase-v0_96/configuration/global.xml        |   160 -
 .../hbase-v0_96/configuration/hbase-log4j.xml   |   142 -
 .../hbase-v0_96/configuration/hbase-policy.xml  |    53 -
 .../hbase-v0_96/configuration/hbase-site.xml    |   365 -
 app-packages/hbase-v0_96/ganglia_metrics.json   |    38 -
 app-packages/hbase-v0_96/jmx_metrics.json       |    56 -
 app-packages/hbase-v0_96/metainfo.xml           |   105 -
 .../hbase-0.96.1-hadoop2-bin.tar.gz.REPLACE     |    16 -
 .../hbase-v0_96/package/scripts/__init__.py     |    19 -
 .../hbase-v0_96/package/scripts/functions.py    |    40 -
 .../hbase-v0_96/package/scripts/hbase.py        |   125 -
 .../hbase-v0_96/package/scripts/hbase_client.py |    43 -
 .../hbase-v0_96/package/scripts/hbase_master.py |    63 -
 .../package/scripts/hbase_regionserver.py       |    66 -
 .../package/scripts/hbase_service.py            |    45 -
 .../hbase-v0_96/package/scripts/params.py       |   109 -
 .../package/scripts/status_params.py            |    26 -
 ...-metrics2-hbase.properties-GANGLIA-MASTER.j2 |    62 -
 ...doop-metrics2-hbase.properties-GANGLIA-RS.j2 |    62 -
 .../package/templates/hbase-env.sh.j2           |    81 -
 .../package/templates/hbase_client_jaas.conf.j2 |    22 -
 .../package/templates/hbase_master_jaas.conf.j2 |    25 -
 .../templates/hbase_regionserver_jaas.conf.j2   |    25 -
 .../package/templates/regionservers.j2          |    20 -
 app-packages/hbase-v0_96/resources.json         |    19 -
 app-packages/hbase/README.txt                   |    75 +
 app-packages/hbase/appConfig.json               |    67 +
 app-packages/hbase/configuration/global.xml     |   160 +
 .../hbase/configuration/hbase-log4j.xml         |   142 +
 .../hbase/configuration/hbase-policy.xml        |    53 +
 app-packages/hbase/configuration/hbase-site.xml |   365 +
 app-packages/hbase/get-hbase-site.sh            |    24 +
 app-packages/hbase/jmx_metrics.json             |    56 +
 app-packages/hbase/metainfo.xml                 |   107 +
 app-packages/hbase/package/scripts/__init__.py  |    19 +
 app-packages/hbase/package/scripts/functions.py |    40 +
 app-packages/hbase/package/scripts/hbase.py     |   125 +
 .../hbase/package/scripts/hbase_client.py       |    43 +
 .../hbase/package/scripts/hbase_master.py       |    63 +
 .../hbase/package/scripts/hbase_regionserver.py |    66 +
 .../hbase/package/scripts/hbase_service.py      |    45 +
 app-packages/hbase/package/scripts/params.py    |   109 +
 .../hbase/package/scripts/status_params.py      |    26 +
 ...-metrics2-hbase.properties-GANGLIA-MASTER.j2 |    62 +
 ...doop-metrics2-hbase.properties-GANGLIA-RS.j2 |    62 +
 .../hbase/package/templates/hbase-env.sh.j2     |    81 +
 .../package/templates/hbase_client_jaas.conf.j2 |    22 +
 .../package/templates/hbase_master_jaas.conf.j2 |    25 +
 .../templates/hbase_regionserver_jaas.conf.j2   |    25 +
 .../hbase/package/templates/regionservers.j2    |    20 +
 app-packages/hbase/pom.xml                      |   253 +
 app-packages/hbase/resources.json               |    21 +
 app-packages/hbase/src/assembly/hbase.xml       |    72 +
 .../hbase/HBaseAgentCommandTestBase.groovy      |    52 +
 .../slider/funtest/hbase/HBaseBasicIT.groovy    |   114 +
 .../funtest/hbase/HBaseMonitorSSLIT.groovy      |    73 +
 .../hbase/StubToForceGroovyTestsToCompile.java  |    22 +
 .../test/resources/appConfig_monitor_ssl.json   |    68 +
 .../hbase/src/test/resources/resources.json     |    21 +
 app-packages/storm-v0_91/README.txt             |    33 -
 app-packages/storm-v0_91/appConfig.json         |   126 -
 .../storm-v0_91/configuration/global.xml        |    39 -
 .../storm-v0_91/configuration/storm-site.xml    |   587 -
 app-packages/storm-v0_91/ganglia_metrics.json   |    31 -
 app-packages/storm-v0_91/jmx_metrics.json       |    31 -
 app-packages/storm-v0_91/metainfo.xml           |   142 -
 ...pache-storm-0.9.1.2.1.1.0-237.tar.gz.REPLACE |    16 -
 .../storm-v0_91/package/scripts/drpc_server.py  |    55 -
 .../storm-v0_91/package/scripts/nimbus.py       |    55 -
 .../storm-v0_91/package/scripts/params.py       |    58 -
 .../storm-v0_91/package/scripts/rest_api.py     |    57 -
 .../storm-v0_91/package/scripts/service.py      |    95 -
 .../package/scripts/status_params.py            |    36 -
 .../storm-v0_91/package/scripts/storm.py        |    50 -
 .../storm-v0_91/package/scripts/supervisor.py   |    61 -
 .../storm-v0_91/package/scripts/ui_server.py    |    55 -
 .../storm-v0_91/package/scripts/yaml_config.py  |    69 -
 .../package/templates/config.yaml.j2            |    48 -
 .../package/templates/storm_jaas.conf.j2        |    26 -
 app-packages/storm-v0_91/resources.json         |    31 -
 app-packages/storm/README.txt                   |    37 +
 app-packages/storm/appConfig.json               |   126 +
 app-packages/storm/configuration/global.xml     |    39 +
 app-packages/storm/configuration/storm-site.xml |   587 +
 app-packages/storm/ganglia_metrics.json         |    31 +
 app-packages/storm/jmx_metrics.json             |    31 +
 app-packages/storm/metainfo.xml                 |   145 +
 ...pache-storm-0.9.1.2.1.1.0-237.tar.gz.REPLACE |    16 +
 .../storm/package/scripts/drpc_server.py        |    55 +
 app-packages/storm/package/scripts/nimbus.py    |    55 +
 app-packages/storm/package/scripts/params.py    |    58 +
 app-packages/storm/package/scripts/rest_api.py  |    57 +
 app-packages/storm/package/scripts/service.py   |    95 +
 .../storm/package/scripts/status_params.py      |    36 +
 app-packages/storm/package/scripts/storm.py     |    50 +
 .../storm/package/scripts/supervisor.py         |    61 +
 app-packages/storm/package/scripts/ui_server.py |    55 +
 .../storm/package/scripts/yaml_config.py        |    69 +
 .../storm/package/templates/config.yaml.j2      |    48 +
 .../storm/package/templates/storm_jaas.conf.j2  |    26 +
 app-packages/storm/resources.json               |    31 +
 pom.xml                                         |   472 +-
 slider-agent/conf/agent.ini                     |     2 +
 slider-agent/pom.xml                            |     6 +-
 .../src/main/python/agent/AgentConfig.py        |    24 +
 slider-agent/src/main/python/agent/Constants.py |     3 +
 .../src/main/python/agent/Controller.py         |    55 +-
 .../python/agent/CustomServiceOrchestrator.py   |     2 +
 slider-agent/src/main/python/agent/NetUtil.py   |     8 +-
 .../src/main/python/agent/PythonExecutor.py     |    15 +-
 slider-agent/src/main/python/agent/main.py      |    19 +-
 slider-agent/src/main/python/agent/security.py  |     7 +-
 .../python/resource_management/core/logger.py   |     4 +-
 .../core/providers/package/apt.py               |     4 +-
 .../python/resource_management/core/shell.py    |    19 +-
 .../libraries/functions/__init__.py             |     1 +
 .../libraries/functions/format.py               |     4 +-
 .../libraries/functions/get_port_from_url.py    |    40 +
 .../functions/get_unique_id_and_date.py         |     2 +-
 .../libraries/functions/os_check.py             |   172 +-
 .../libraries/providers/copy_from_local.py      |     2 +-
 .../libraries/providers/execute_hadoop.py       |     2 +-
 .../libraries/script/hook.py                    |    12 +-
 .../libraries/script/script.py                  |    28 +-
 slider-agent/src/main/python/setup.py           |     6 +-
 slider-agent/src/packages/tarball/all.xml       |     3 +-
 .../src/test/python/agent/TestActionQueue.py    |     1 +
 .../src/test/python/agent/TestController.py     |    34 +-
 .../agent/TestCustomServiceOrchestrator.py      |     2 +-
 slider-agent/src/test/python/agent/TestMain.py  |     9 +-
 .../src/test/python/agent/TestNetUtil.py        |     2 +-
 .../src/test/python/agent/TestPythonExecutor.py |    12 +-
 slider-agent/src/test/python/python-wrap        |    40 +
 .../resource_management/TestLibraryFunctions.py |    33 +
 .../resource_management/TestPackageResource.py  |     2 +-
 .../TestPropertiesFileResource.py               |     2 +-
 slider-assembly/pom.xml                         |   198 +-
 slider-assembly/src/conf-hdp/log4j.properties   |    58 +
 slider-assembly/src/conf-hdp/slider-client.xml  |    77 +
 slider-assembly/src/conf/slider-client.xml      |     3 -
 slider-assembly/src/main/scripts/slider         |     1 -
 slider-assembly/src/main/scripts/slider.py      |   190 +
 slider-core/pom.xml                             |   239 +-
 .../java/org/apache/slider/api/StatusKeys.java  |     3 +
 .../org/apache/slider/api/proto/Messages.java   | 13186 +++++++++++++++++
 .../slider/api/proto/SliderClusterAPI.java      |  1043 ++
 .../org/apache/slider/client/SliderClient.java  |   123 +-
 .../slider/client/SliderYarnClientImpl.java     |     3 +-
 .../org/apache/slider/common/SliderKeys.java    |    14 +
 .../apache/slider/common/SliderXmlConfKeys.java |     6 +-
 .../slider/common/tools/ConfigHelper.java       |     1 -
 .../apache/slider/common/tools/Duration.java    |    67 +-
 .../apache/slider/common/tools/SliderUtils.java |    59 +-
 .../core/launch/ClasspathConstructor.java       |    21 +-
 .../core/launch/JavaCommandLineBuilder.java     |     2 +-
 .../slider/core/main/ExitCodeProvider.java      |     2 +-
 .../org/apache/slider/core/main/IrqHandler.java |     2 +-
 .../slider/core/main/LauncherExitCodes.java     |    27 +-
 .../org/apache/slider/core/main/RunService.java |     2 +-
 .../core/main/ServiceLaunchException.java       |    24 +-
 .../slider/core/main/ServiceLauncher.java       |   229 +-
 .../slider/core/main/ServiceShutdownHook.java   |    27 +-
 .../registry/info/CustomRegistryConstants.java  |     2 +-
 .../core/registry/info/RegisteredEndpoint.java  |    44 +-
 .../core/registry/info/ServiceInstanceData.java |     2 +
 .../slider/core/zk/BlockingZKWatcher.java       |     8 +-
 .../apache/slider/core/zk/ZKIntegration.java    |    23 +-
 .../apache/slider/core/zk/ZookeeperUtils.java   |    24 +-
 .../providers/AbstractProviderService.java      |    46 +-
 .../slider/providers/ProviderCompleted.java     |    29 +
 .../providers/ProviderCompletedCallable.java    |    38 +
 .../slider/providers/ProviderService.java       |    16 +-
 .../providers/agent/AgentClientProvider.java    |    45 +-
 .../slider/providers/agent/AgentKeys.java       |    10 +-
 .../providers/agent/AgentLaunchParameter.java   |   130 +
 .../providers/agent/AgentProviderService.java   |   282 +-
 .../slider/providers/agent/AgentRoles.java      |    18 +-
 .../apache/slider/providers/agent/Command.java  |    13 +-
 .../slider/providers/agent/CommandResult.java   |    16 +-
 .../providers/agent/ComponentInstanceState.java |    24 +
 .../slider/providers/agent/ContainerState.java  |    41 +
 .../providers/agent/HeartbeatMonitor.java       |   116 +
 .../agent/application/metadata/Application.java |   121 +
 .../agent/application/metadata/Metainfo.java    |    17 +-
 .../application/metadata/MetainfoParser.java    |    10 +-
 .../agent/application/metadata/Service.java     |   121 -
 .../slideram/SliderAMClientProvider.java        |     7 +-
 .../slideram/SliderAMProviderService.java       |    22 +-
 .../server/appmaster/AMViewForProviders.java    |    27 +
 .../server/appmaster/RoleLaunchService.java     |   147 +-
 .../server/appmaster/SliderAppMaster.java       |   174 +-
 .../slider/server/appmaster/state/AppState.java |    22 +-
 .../appmaster/state/ContainerPriority.java      |     8 +-
 .../appmaster/state/OutstandingRequest.java     |     7 +-
 .../server/appmaster/state/RoleHistory.java     |     8 +
 .../server/appmaster/web/AgentService.java      |    53 +
 .../server/appmaster/web/SliderAMWebApp.java    |     5 +-
 .../server/appmaster/web/SliderAmIpFilter.java  |    21 +-
 .../slider/server/appmaster/web/WebAppApi.java  |     9 +-
 .../server/appmaster/web/WebAppApiImpl.java     |    16 +-
 .../appmaster/web/rest/AMWebServices.java       |     7 +-
 .../server/appmaster/web/rest/RestPaths.java    |     7 +-
 .../appmaster/web/rest/agent/AgentResource.java |    25 +-
 .../appmaster/web/rest/agent/AgentWebApp.java   |   203 +
 .../web/rest/agent/AgentWebServices.java        |    40 +
 .../web/rest/publisher/PublisherResource.java   |    14 +
 .../server/exec/ApplicationEventHandler.java    |    29 -
 .../slider/server/exec/RunLongLivedApp.java     |   439 -
 .../server/services/curator/CuratorService.java |     2 +-
 .../server/services/curator/CuratorUriSpec.java |     2 +-
 .../services/security/CertificateManager.java   |   257 +
 .../server/services/security/SecurityUtils.java |   209 +
 .../services/security/SignCertResponse.java     |    67 +
 .../server/services/security/SignMessage.java   |    54 +
 .../utility/AbstractSliderLaunchedService.java  |    15 +-
 .../server/services/utility/ClosingService.java |    59 -
 .../utility/CompoundLaunchedService.java        |   134 -
 .../services/utility/CompoundService.java       |   115 -
 .../server/services/utility/EventCallback.java  |    25 -
 .../services/utility/EventNotifyingService.java |    63 -
 .../services/utility/ForkedProcessService.java  |   233 -
 .../LaunchedWorkflowCompositeService.java       |   113 +
 .../slider/server/services/utility/Parent.java  |    39 -
 .../services/utility/PatternValidator.java      |    14 +-
 .../server/services/utility/RpcService.java     |    65 -
 .../utility/SecurityCheckerService.java         |    40 -
 .../services/utility/SequenceService.java       |   243 -
 .../services/utility/SliderServiceUtils.java    |    29 -
 .../server/services/utility/WebAppService.java  |     2 +-
 .../AbstractWorkflowExecutorService.java        |   110 +
 .../services/workflow/ClosingService.java       |    91 +
 .../services/workflow/ForkedProcessService.java |   285 +
 .../services/workflow/LongLivedProcess.java     |   559 +
 .../LongLivedProcessLifecycleEvent.java         |    41 +
 .../server/services/workflow/ServiceParent.java |    44 +
 .../workflow/ServiceTerminatingCallable.java    |    92 +
 .../workflow/ServiceTerminatingRunnable.java    |    72 +
 .../services/workflow/ServiceThreadFactory.java |   100 +
 .../workflow/WorkflowCallbackService.java       |   111 +
 .../workflow/WorkflowCompositeService.java      |   151 +
 .../services/workflow/WorkflowRpcService.java   |    76 +
 .../workflow/WorkflowSequenceService.java       |   300 +
 .../server/services/workflow/package-info.java  |   172 +
 .../src/main/proto/SliderClusterMessages.proto  |     4 +
 .../src/main/proto/SliderClusterProtocol.proto  |     4 +
 .../providers/slideram/instance/appconf.json    |     5 +-
 .../main/resources/webapps/slideragent/.keep    |     0
 .../resources/webapps/static/yarn.dt.plugins.js |     4 +-
 .../test_command_log/appConfig.json             |     8 +-
 .../test_command_log/appConfig_fast_no_reg.json |    29 +
 .../test_command_log/appConfig_no_hb.json       |    29 +
 .../test_command_log/cmd_log_app_pkg.zip        |   Bin 6676 -> 0 bytes
 .../test_command_log/configuration/cl-site.xml  |    34 -
 .../app_packages/test_command_log/metainfo.xml  |    54 -
 .../package/files/command_log_10.tar            |   Bin 2560 -> 0 bytes
 .../test_command_log/package/scripts/cl.py      |    89 -
 .../test_command_log/package/scripts/params.py  |    31 -
 .../package/templates/operations.log.j2         |    22 -
 .../test_command_log/resources_no_role.json     |    15 +
 .../agent/AgentMiniClusterTestBase.groovy       |    43 +-
 .../agent/actions/TestActionExists.groovy       |    12 +-
 .../slider/agent/actions/TestActionList.groovy  |     6 +-
 .../standalone/TestStandaloneAgentAM.groovy     |     2 +-
 .../standalone/TestStandaloneRegistryAM.groovy  |     2 +-
 .../common/tools/TestZKIntegration.groovy       |    53 +
 .../slider/providers/agent/AgentTestBase.groovy |    44 +-
 .../providers/agent/AgentTestUtils.groovy       |    13 +-
 .../agent/TestAgentAMManagementWS.groovy        |    49 +-
 .../slider/providers/agent/TestAgentEcho.groovy |     6 +-
 .../providers/agent/TestBuildBasicAgent.groovy  |   129 +-
 .../curator/TestRegistryRestResources.groovy    |     3 +-
 .../model/appstate/TestMockRMOperations.groovy  |     2 +-
 .../model/mock/MockProviderService.groovy       |    20 +-
 .../publisher/TestPublisherRestResources.groovy |    15 +-
 .../view/TestClusterSpecificationBlock.groovy   |     2 +-
 .../web/view/TestContainerStatsBlock.groovy     |     2 +-
 .../appmaster/web/view/TestIndexBlock.groovy    |     2 +-
 .../server/services/utility/MockService.groovy  |    72 -
 .../services/utility/TestCompoundService.groovy |   159 -
 .../services/utility/TestMockService.groovy     |    94 -
 .../services/utility/TestSequenceService.groovy |   142 -
 .../apache/slider/test/SliderTestUtils.groovy   |     8 +-
 .../slider/test/YarnMiniClusterTestBase.groovy  |    54 +-
 .../test/YarnZKMiniClusterTestBase.groovy       |     4 +-
 .../slider/common/tools/TestSliderUtils.java    |    23 +-
 .../agent/TestAgentClientProvider.java          |    40 +-
 .../agent/TestAgentLaunchParameter.java         |    76 +
 .../agent/TestAgentProviderService.java         |    68 +-
 .../providers/agent/TestHeartbeatMonitor.java   |   136 +
 .../metadata/MetainfoParserTest.java            |    15 +-
 .../appmaster/web/TestSliderAmFilter.java       |    14 +-
 .../web/rest/agent/TestAMAgentWebServices.java  |   237 +-
 .../management/TestAMManagementWebServices.java |    67 +-
 .../publisher/TestAgentProviderService.java     |     5 +-
 .../security/TestCertificateManager.java        |    60 +
 .../services/workflow/EndOfServiceWaiter.java   |    56 +
 .../server/services/workflow/MockService.java   |    80 +
 .../workflow/ParentWorkflowTestBase.java        |    70 +
 .../workflow/ProcessCommandFactory.java         |    87 +
 .../services/workflow/SimpleRunnable.java       |    46 +
 .../services/workflow/TestLongLivedProcess.java |   161 +
 .../workflow/TestWorkflowClosingService.java    |   116 +
 .../workflow/TestWorkflowCompositeService.java  |   113 +
 .../workflow/TestWorkflowExecutorService.java   |    61 +
 .../TestWorkflowForkedProcessService.java       |   141 +
 .../workflow/TestWorkflowRpcService.java        |   107 +
 .../workflow/TestWorkflowSequenceService.java   |   151 +
 .../TestWorkflowServiceTerminatingRunnable.java |    64 +
 .../workflow/WorkflowServiceTestBase.java       |   138 +
 .../org/apache/slider/tools/TestUtility.java    |    71 +
 slider-core/src/test/python/agent.py            |    17 +-
 slider-core/src/test/python/agent/main.py       |    29 +-
 slider-core/src/test/python/appdef_1.zip        |   Bin 972 -> 0 bytes
 slider-core/src/test/python/echo.py             |    21 +-
 slider-core/src/test/python/metainfo.xml        |    68 +-
 .../src/test/resources/example-slider-test.xml  |     4 +-
 .../org/apache/slider/common/tools/test.zip     |   Bin 1273 -> 0 bytes
 .../slider/common/tools/test/metainfo.txt       |    16 +
 .../slider/common/tools/test/metainfo.xml       |    95 +
 .../slider/common/tools/test/someOtherFile.txt  |    16 +
 .../slider/common/tools/test/someOtherFile.xml  |    16 +
 .../agent/application/metadata/metainfo.xml     |   136 +-
 slider-funtest/pom.xml                          |   123 +-
 .../framework/AgentCommandTestBase.groovy       |   237 +
 .../funtest/framework/AgentUploads.groovy       |    65 +
 .../funtest/framework/CommandTestBase.groovy    |    16 +-
 .../funtest/framework/FileUploader.groovy       |   151 +
 .../funtest/framework/FuntestProperties.groovy  |    12 +-
 .../slider/funtest/framework/SliderShell.groovy |    68 +-
 .../slider/funtest/framework/SudoClosure.groovy |    54 +
 .../accumulo/StubToForceGroovySrcToCompile.java |    22 +
 .../basic/TestClusterConnectivity.groovy        |   109 +
 .../lifecycle/AgentCommandTestBase.groovy       |   134 -
 .../lifecycle/TestAgentClusterLifecycle.groovy  |    84 +-
 .../funtest/lifecycle/TestAgentFailures.groovy  |   104 +
 .../funtest/lifecycle/TestAgentFailures2.groovy |   104 +
 .../lifecycle/TestAppsThroughAgent.groovy       |   133 +-
 .../lifecycle/TestClusterBuildDestroy.groovy    |     6 +-
 slider-install/README.md                        |   102 +
 slider-install/pom.xml                          |   111 +
 slider-install/src/main/bash/slider-client.xml  |    83 +
 slider-install/src/main/bash/slider_destroy     |    64 +
 slider-install/src/main/bash/slider_setup       |   173 +
 slider-install/src/main/bash/slider_setup.conf  |    60 +
 .../accumulo/accumulo-funtests/pom.xml          |    20 +-
 .../slider/providers/accumulo/TestStub.groovy   |    32 +
 .../accumulo/slider-accumulo-provider/pom.xml   |    47 +-
 .../accumulo/AccumuloProviderService.java       |    32 +-
 .../providers/accumulo/AccumuloTestBase.groovy  |    98 +-
 .../live/TestAccCorrectInstanceName.groovy      |     2 +-
 .../accumulo/live/TestAccFreezeThaw.groovy      |     4 +-
 .../accumulo/live/TestAccLiveHDFSArchive.groovy |     2 +-
 .../live/TestAccLiveLocalArchive.groovy         |     2 +-
 .../accumulo/live/TestAccM1T1GC1Mon1.groovy     |     2 +-
 .../accumulo/live/TestAccM2T2GC1Mon1.groovy     |     2 +-
 .../accumulo/live/TestAccumuloAMWebApp.groovy   |     2 +-
 slider-providers/hbase/hbase-funtests/pom.xml   |    39 +-
 .../slider/providers/hbase/TestStub.groovy      |    32 +
 .../hbase/funtest/HBaseCommandTestBase.groovy   |     1 +
 .../hbase/slider-hbase-provider/pom.xml         |    35 +-
 .../providers/hbase/HBaseProviderService.java   |    22 +-
 .../slider/providers/hbase/HBaseRoles.java      |     3 +-
 .../failures/TestKilledHBaseAM.groovy           |     6 +-
 .../flexing/TestClusterFlex0To1.groovy          |     2 +-
 .../minicluster/live/TestTwoLiveClusters.groovy |     2 +-
 src/docs/reports/2014-06-04-report.txt          |    67 +
 src/site/markdown/architecture/architecture.md  |   142 -
 src/site/markdown/architecture/index.md         |    27 -
 src/site/markdown/architecture/rolehistory.md   |  1010 --
 src/site/markdown/client-configuration.md       |   310 -
 src/site/markdown/configuration/core.md         |   407 -
 .../example-app_configuration-resolved.json     |    42 -
 .../example-app_configuration.json              |    25 -
 .../markdown/configuration/example-empty.json   |     8 -
 .../configuration/example-internal.json         |    21 -
 .../example-overridden-resolved.json            |    25 -
 .../configuration/example-overridden.json       |    23 -
 .../configuration/example-resources.json        |    25 -
 .../markdown/configuration/index-markdown.md    |    30 -
 src/site/markdown/configuration/index.md        |    38 -
 .../markdown/configuration/original-hbase.json  |   139 -
 .../markdown/configuration/proposed-hbase.json  |   273 -
 src/site/markdown/configuration/redesign.md     |   478 -
 .../configuration/resolved-resources.json       |    22 -
 .../markdown/configuration/specification.md     |   512 -
 src/site/markdown/debugging.md                  |    92 -
 src/site/markdown/developing/building.md        |   374 -
 .../markdown/developing/functional_tests.md     |   416 -
 src/site/markdown/developing/index.md           |    35 -
 src/site/markdown/developing/manual_testing.md  |    53 -
 src/site/markdown/developing/releasing.md       |   195 -
 src/site/markdown/developing/testing.md         |   182 -
 src/site/markdown/examples.md                   |   159 -
 src/site/markdown/exitcodes.md                  |   161 -
 src/site/markdown/getting_started.md            |   509 -
 src/site/markdown/index.md                      |    83 -
 src/site/markdown/manpage.md                    |   483 -
 .../registry/a_YARN_service_registry.md         |   227 -
 src/site/markdown/registry/index.md             |    47 -
 .../registry/initial_registry_design.md         |   110 -
 .../markdown/registry/p2p_service_registries.md |    99 -
 src/site/markdown/registry/references.md        |    49 -
 src/site/markdown/registry/registry-model.md    |    75 -
 .../service_registry_end_to_end_scenario.md     |   156 -
 ...lication_registration_and_binding_problem.md |   145 -
 .../markdown/release_notes/release-0.22.0.md    |    48 -
 .../markdown/release_notes/release-0.30.0.md    |    36 -
 src/site/markdown/security.md                   |   197 -
 .../slider_specs/application_configuration.md   |    82 -
 .../slider_specs/application_definition.md      |   182 -
 .../application_instance_configuration.md       |   104 -
 .../markdown/slider_specs/application_needs.md  |   140 -
 .../slider_specs/application_package.md         |   147 -
 .../slider_specs/canonical_scenarios.md         |   165 -
 .../slider_specs/creating_app_definitions.md    |   123 -
 src/site/markdown/slider_specs/index.md         |    53 -
 .../slider_specs/resource_specification.md      |    53 -
 .../slider_specs/writing_app_command_scripts.md |   211 -
 src/site/markdown/specification/cli-actions.md  |   675 -
 src/site/markdown/specification/index.md        |    41 -
 src/site/markdown/specification/slider-model.md |   286 -
 src/site/markdown/troubleshooting.md            |   154 -
 src/site/resources/hoya_am_architecture.png     |   Bin 137154 -> 0 bytes
 .../resources/images/app_config_folders_01.png  |   Bin 21050 -> 0 bytes
 .../resources/images/app_package_sample_04.png  |   Bin 67927 -> 0 bytes
 src/site/resources/images/image_0.png           |   Bin 194914 -> 0 bytes
 src/site/resources/images/image_1.png           |   Bin 77882 -> 0 bytes
 src/site/resources/images/managed_client.png    |   Bin 71352 -> 0 bytes
 src/site/resources/images/slider-container.png  |   Bin 70265 -> 0 bytes
 src/site/resources/images/unmanaged_client.png  |   Bin 58813 -> 0 bytes
 src/site/site.xml                               |    63 -
 src/test/clusters/c6401/slider/log4j.properties |    53 +
 .../clusters/c6401/slider/slider-client.xml     |    72 +
 .../clusters/morzine/slider/log4j.properties    |    53 +
 .../clusters/morzine/slider/slider-client.xml   |    76 +
 .../clusters/offline/slider/log4j.properties    |    30 -
 .../clusters/offline/slider/slider-client.xml   |     7 +-
 .../clusters/remote/slider/log4j.properties     |    30 -
 .../clusters/remote/slider/slider-client.xml    |     5 -
 .../clusters/sandbox/slider/log4j.properties    |    30 -
 .../clusters/sandbox/slider/slider-client.xml   |     5 -
 src/test/clusters/ubuntu-secure/operations.md   |    36 +-
 .../ubuntu-secure/slider/log4j.properties       |    26 -
 537 files changed, 33311 insertions(+), 19979 deletions(-)
----------------------------------------------------------------------



[11/50] [abbrv] git commit: SLIDER-207 TestClusterConnectivity fails on Jenkins

Posted by sm...@apache.org.
SLIDER-207 TestClusterConnectivity fails on Jenkins


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

Branch: refs/heads/master
Commit: 0f9344bdae038174f0faf6a34cc1686dad0f3840
Parents: 4933cd8
Author: Steve Loughran <st...@apache.org>
Authored: Thu Jul 3 09:28:08 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Jul 3 09:28:08 2014 +0100

----------------------------------------------------------------------
 .../basic/TestClusterConnectivity.groovy        | 33 ++++++++++++++------
 1 file changed, 23 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0f9344bd/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
index 8436dc9..b9d768a 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/basic/TestClusterConnectivity.groovy
@@ -26,17 +26,32 @@ import org.apache.slider.client.SliderYarnClientImpl
 import org.apache.slider.common.SliderXmlConfKeys
 import org.apache.slider.core.zk.ZookeeperUtils
 import org.apache.slider.funtest.framework.CommandTestBase
+import org.junit.BeforeClass
 import org.junit.Test
 
 @Slf4j
+/**
+ * Test basic connectivity with the target cluster, including 
+ * HDFS, YARN and ZK
+ */
 class TestClusterConnectivity extends CommandTestBase {
 
+
+  public static final int CONNECT_TIMEOUT = 2000
+
+  @BeforeClass
+  public static void setup() {
+    assumeFunctionalTestsEnabled()
+  }
+  
   @Test
   public void testFileSystemUp() throws Throwable {
 
     def fs = clusterFS
     def status = fs.listStatus(new Path("/"))
-    status.each {it -> log.info("${it.path} = ${it}")}
+    status.each {
+      log.info("${it.path} = ${it}")
+    }
     
   }
 
@@ -45,7 +60,9 @@ class TestClusterConnectivity extends CommandTestBase {
     def quorum = SLIDER_CONFIG.getTrimmed(SliderXmlConfKeys.REGISTRY_ZK_QUORUM)
     assert quorum
     def tuples = ZookeeperUtils.splitToHostsAndPortsStrictly(quorum);
-    tuples.each {it -> telnet(it.getHostText(), it.getPort())}
+    tuples.each {
+      telnet(it.hostText, it.port)
+    }
     
   }
 
@@ -79,18 +96,14 @@ class TestClusterConnectivity extends CommandTestBase {
     assert host != ""
     assert port != 0
     try {
-      def socket = new Socket(host, port);
+      def socket = new Socket();
+      def addr = new InetSocketAddress(host, port)
+      socket.connect(addr, CONNECT_TIMEOUT)
+      socket.close()
     } catch (IOException e) {
       throw NetUtils.wrapException(host, port, "localhost", 0, e)
     }
-/*
 
-    socket.withStreams { input, output ->
-      output << "echo testing ...\n"
-      def buffer = input.newReader().readLine()
-      println "response = ${buffer}"
-*/
-    
   }
   
 }


[49/50] [abbrv] git commit: Merge branch 'release/slider-0.40' into develop

Posted by sm...@apache.org.
Merge branch 'release/slider-0.40' into develop


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

Branch: refs/heads/develop
Commit: 454f5cc753348c3a398493108067c6e52d3aa154
Parents: bedf1c1 e09a6a8
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Thu Jul 10 21:15:00 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Thu Jul 10 21:15:00 2014 -0700

----------------------------------------------------------------------
 app-packages/accumulo/pom.xml                              | 2 +-
 app-packages/command-logger/application-pkg/pom.xml        | 2 +-
 app-packages/command-logger/slider-pkg/pom.xml             | 2 +-
 pom.xml                                                    | 2 +-
 slider-agent/pom.xml                                       | 2 +-
 slider-assembly/pom.xml                                    | 2 +-
 slider-core/pom.xml                                        | 2 +-
 slider-funtest/pom.xml                                     | 2 +-
 slider-install/pom.xml                                     | 2 +-
 slider-providers/accumulo/accumulo-funtests/pom.xml        | 2 +-
 slider-providers/accumulo/slider-accumulo-provider/pom.xml | 2 +-
 slider-providers/hbase/hbase-funtests/pom.xml              | 2 +-
 slider-providers/hbase/slider-hbase-provider/pom.xml       | 2 +-
 13 files changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------



[27/50] [abbrv] SLIDER-181. Storm App Package (minor config changes and folder name changes)

Posted by sm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/appConfig.json
----------------------------------------------------------------------
diff --git a/app-packages/storm/appConfig.json b/app-packages/storm/appConfig.json
new file mode 100644
index 0000000..6d6aa3a
--- /dev/null
+++ b/app-packages/storm/appConfig.json
@@ -0,0 +1,126 @@
+{
+  "schema": "http://example.org/specification/v2.0.0",
+  "metadata": {
+  },
+  "global": {
+    "application.def": "/slider/storm_v091.zip",
+    "config_types": "storm-site",
+    "java_home": "/usr/jdk64/jdk1.7.0_45",
+    "package_list": "files/apache-storm-0.9.1.2.1.1.0-237.tar.gz",
+    "create.default.zookeeper.node": "true",
+    "site.global.app_user": "yarn",
+    "site.global.app_root": "${AGENT_WORK_ROOT}/app/install/apache-storm-0.9.1.2.1.1.0-237",
+    "site.global.user_group": "hadoop",
+    "site.global.security_enabled": "false",
+    "site.global.ganglia_server_host": "${NN_HOST}",
+    "site.global.ganglia_server_id": "Application2",
+    "site.global.ganglia_enabled":"true",
+    "site.global.ganglia_server_port": "8668",
+    "site.global.rest_api_port": "${STORM_REST_API.ALLOCATED_PORT}",
+    "site.global.rest_api_admin_port": "${STORM_REST_API.ALLOCATED_PORT}",
+    "site.storm-site.topology.tuple.serializer": "backtype.storm.serialization.types.ListDelegateSerializer",
+    "site.storm-site.topology.workers": "1",
+    "site.storm-site.drpc.worker.threads": "64",
+    "site.storm-site.storm.zookeeper.servers": "['${ZK_HOST}']",
+    "site.storm-site.supervisor.heartbeat.frequency.secs": "5",
+    "site.storm-site.topology.executor.send.buffer.size": "1024",
+    "site.storm-site.drpc.childopts": "-Xmx768m",
+    "site.storm-site.nimbus.thrift.port": "${NIMBUS.ALLOCATED_PORT}",
+    "site.storm-site.storm.zookeeper.retry.intervalceiling.millis": "30000",
+    "site.storm-site.storm.local.dir": "${AGENT_WORK_ROOT}/app/tmp/storm",
+    "site.storm-site.topology.receiver.buffer.size": "8",
+    "site.storm-site.storm.messaging.netty.client_worker_threads": "1",
+    "site.storm-site.transactional.zookeeper.root": "/transactional",
+    "site.storm-site.drpc.request.timeout.secs": "600",
+    "site.storm-site.topology.skip.missing.kryo.registrations": "false",
+    "site.storm-site.worker.heartbeat.frequency.secs": "1",
+    "site.storm-site.zmq.hwm": "0",
+    "site.storm-site.storm.zookeeper.connection.timeout": "15000",
+    "site.storm-site.topology.max.error.report.per.interval": "5",
+    "site.storm-site.storm.messaging.netty.server_worker_threads": "1",
+    "site.storm-site.supervisor.worker.start.timeout.secs": "120",
+    "site.storm-site.zmq.threads": "1",
+    "site.storm-site.topology.acker.executors": "null",
+    "site.storm-site.storm.local.mode.zmq": "false",
+    "site.storm-site.topology.max.task.parallelism": "null",
+    "site.storm-site.storm.zookeeper.port": "2181",
+    "site.storm-site.nimbus.childopts": "-Xmx1024m -javaagent:${AGENT_WORK_ROOT}/app/install/apache-storm-0.9.1.2.1.1.0-237/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=${NN_HOST},port=8668,wireformat31x=true,mode=multicast,config=${AGENT_WORK_ROOT}/app/install/apache-storm-0.9.1.2.1.1.0-237/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Nimbus_JVM",
+    "site.storm-site.worker.childopts": "-Xmx768m -javaagent:${AGENT_WORK_ROOT}/app/install/apache-storm-0.9.1.2.1.1.0-237/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=${NN_HOST},port=8668,wireformat31x=true,mode=multicast,config=${AGENT_WORK_ROOT}/app/install/apache-storm-0.9.1.2.1.1.0-237/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Worker_%ID%_JVM",
+    "site.storm-site.drpc.queue.size": "128",
+    "site.storm-site.storm.zookeeper.retry.times": "5",
+    "site.storm-site.nimbus.monitor.freq.secs": "10",
+    "site.storm-site.storm.cluster.mode": "distributed",
+    "site.storm-site.dev.zookeeper.path": "${AGENT_WORK_ROOT}/app/tmp/dev-storm-zookeeper",
+    "site.storm-site.drpc.invocations.port": "0",
+    "site.storm-site.storm.zookeeper.root": "${DEF_ZK_PATH}",
+    "site.storm-site.logviewer.childopts": "-Xmx128m",
+    "site.storm-site.transactional.zookeeper.port": "null",
+    "site.storm-site.topology.worker.childopts": "null",
+    "site.storm-site.topology.max.spout.pending": "null",
+    "site.storm-site.nimbus.cleanup.inbox.freq.secs": "600",
+    "site.storm-site.storm.messaging.netty.min_wait_ms": "100",
+    "site.storm-site.nimbus.task.timeout.secs": "30",
+    "site.storm-site.nimbus.thrift.max_buffer_size": "1048576",
+    "site.storm-site.topology.sleep.spout.wait.strategy.time.ms": "1",
+    "site.storm-site.topology.optimize": "true",
+    "site.storm-site.nimbus.reassign": "true",
+    "site.storm-site.storm.messaging.transport": "backtype.storm.messaging.netty.Context",
+    "site.storm-site.logviewer.appender.name": "A1",
+    "site.storm-site.nimbus.host": "${NIMBUS_HOST}",
+    "site.storm-site.ui.port": "${STORM_UI_SERVER.ALLOCATED_PORT}",
+    "site.storm-site.supervisor.slots.ports": "[${SUPERVISOR.ALLOCATED_PORT}]",
+    "site.storm-site.nimbus.file.copy.expiration.secs": "600",
+    "site.storm-site.supervisor.monitor.frequency.secs": "3",
+    "site.storm-site.transactional.zookeeper.servers": "null",
+    "site.storm-site.zmq.linger.millis": "5000",
+    "site.storm-site.topology.error.throttle.interval.secs": "10",
+    "site.storm-site.topology.worker.shared.thread.pool.size": "4",
+    "site.storm-site.java.library.path": "/usr/local/lib:/opt/local/lib:/usr/lib",
+    "site.storm-site.topology.spout.wait.strategy": "backtype.storm.spout.SleepSpoutWaitStrategy",
+    "site.storm-site.task.heartbeat.frequency.secs": "3",
+    "site.storm-site.topology.transfer.buffer.size": "1024",
+    "site.storm-site.storm.zookeeper.session.timeout": "20000",
+    "site.storm-site.topology.executor.receive.buffer.size": "1024",
+    "site.storm-site.topology.stats.sample.rate": "0.05",
+    "site.storm-site.topology.fall.back.on.java.serialization": "true",
+    "site.storm-site.supervisor.childopts": "-Xmx256m -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.port=0 -javaagent:${AGENT_WORK_ROOT}/app/install/apache-storm-0.9.1.2.1.1.0-237/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=${NN_HOST},port=8668,wireformat31x=true,mode=multicast,config=${AGENT_WORK_ROOT}/app/install/apache-storm-0.9.1.2.1.1.0-237/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Supervisor_JVM",
+    "site.storm-site.topology.enable.message.timeouts": "true",
+    "site.storm-site.storm.messaging.netty.max_wait_ms": "1000",
+    "site.storm-site.nimbus.topology.validator": "backtype.storm.nimbus.DefaultTopologyValidator",
+    "site.storm-site.nimbus.supervisor.timeout.secs": "60",
+    "site.storm-site.topology.disruptor.wait.strategy": "com.lmax.disruptor.BlockingWaitStrategy",
+    "site.storm-site.nimbus.inbox.jar.expiration.secs": "3600",
+    "site.storm-site.drpc.port": "0",
+    "site.storm-site.topology.kryo.factory": "backtype.storm.serialization.DefaultKryoFactory",
+    "site.storm-site.storm.zookeeper.retry.interval": "1000",
+    "site.storm-site.storm.messaging.netty.max_retries": "30",
+    "site.storm-site.topology.tick.tuple.freq.secs": "null",
+    "site.storm-site.supervisor.enable": "true",
+    "site.storm-site.nimbus.task.launch.secs": "120",
+    "site.storm-site.task.refresh.poll.secs": "10",
+    "site.storm-site.topology.message.timeout.secs": "30",
+    "site.storm-site.storm.messaging.netty.buffer_size": "5242880",
+    "site.storm-site.topology.state.synchronization.timeout.secs": "60",
+    "site.storm-site.supervisor.worker.timeout.secs": "30",
+    "site.storm-site.topology.trident.batch.emit.interval.millis": "500",
+    "site.storm-site.topology.builtin.metrics.bucket.size.secs": "60",
+    "site.storm-site.storm.thrift.transport": "backtype.storm.security.auth.SimpleTransportPlugin",
+    "site.storm-site.logviewer.port": "0",
+    "site.storm-site.topology.debug": "false"
+  },
+  "components": {
+    "slider-appmaster": {
+      "jvm.heapsize": "256M"
+    },
+    "NIMBUS": {
+    },
+    "STORM_REST_API": {
+    },
+    "STORM_UI_SERVER": {
+    },
+    "DRPC_SERVER": {
+    },
+    "SUPERVISOR": {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/configuration/global.xml
----------------------------------------------------------------------
diff --git a/app-packages/storm/configuration/global.xml b/app-packages/storm/configuration/global.xml
new file mode 100644
index 0000000..5cc9170
--- /dev/null
+++ b/app-packages/storm/configuration/global.xml
@@ -0,0 +1,39 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+/**
+ * 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.
+ */
+-->
+
+<configuration>
+  <property>
+    <name>storm_user</name>
+    <value>storm</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm_log_dir</name>
+    <value>/var/log/storm</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm_pid_dir</name>
+    <value>/var/run/storm</value>
+    <description></description>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/configuration/storm-site.xml
----------------------------------------------------------------------
diff --git a/app-packages/storm/configuration/storm-site.xml b/app-packages/storm/configuration/storm-site.xml
new file mode 100644
index 0000000..6eca8f9
--- /dev/null
+++ b/app-packages/storm/configuration/storm-site.xml
@@ -0,0 +1,587 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+/**
+ * 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.
+ */
+-->
+
+<configuration>
+  <property>
+    <name>java.library.path</name>
+    <value>/usr/local/lib:/opt/local/lib:/usr/lib</value>
+    <description>This value is passed to spawned JVMs (e.g., Nimbus, Supervisor, and Workers)
+       for the java.library.path value. java.library.path tells the JVM where
+       to look for native libraries. It is necessary to set this config correctly since
+       Storm uses the ZeroMQ and JZMQ native libs. </description>
+  </property>
+  <property>
+    <name>storm.local.dir</name>
+    <value>/hadoop/storm</value>
+    <description>A directory on the local filesystem used by Storm for any local
+       filesystem usage it needs. The directory must exist and the Storm daemons must
+       have permission to read/write from this location.</description>
+  </property>
+  <property>
+    <name>storm.zookeeper.servers</name>
+    <value>['localhost']</value>
+    <description>A list of hosts of ZooKeeper servers used to manage the cluster.</description>
+  </property>
+  <property>
+    <name>storm.zookeeper.port</name>
+    <value>2181</value>
+    <description>The port Storm will use to connect to each of the ZooKeeper servers.</description>
+  </property>
+  <property>
+    <name>storm.zookeeper.root</name>
+    <value>/storm</value>
+    <description>The root location at which Storm stores data in ZooKeeper.</description>
+  </property>
+  <property>
+    <name>storm.zookeeper.session.timeout</name>
+    <value>20000</value>
+    <description>The session timeout for clients to ZooKeeper.</description>
+  </property>
+  <property>
+    <name>storm.zookeeper.connection.timeout</name>
+    <value>15000</value>
+    <description>The connection timeout for clients to ZooKeeper.</description>
+  </property>
+  <property>
+    <name>storm.zookeeper.retry.times</name>
+    <value>5</value>
+    <description>The number of times to retry a Zookeeper operation.</description>
+  </property>
+  <property>
+    <name>storm.zookeeper.retry.interval</name>
+    <value>1000</value>
+    <description>The interval between retries of a Zookeeper operation.</description>
+  </property>
+  <property>
+    <name>storm.zookeeper.retry.intervalceiling.millis</name>
+    <value>30000</value>
+    <description>The ceiling of the interval between retries of a Zookeeper operation.</description>
+  </property>
+  <property>
+    <name>storm.cluster.mode</name>
+    <value>distributed</value>
+    <description>The mode this Storm cluster is running in. Either "distributed" or "local".</description>
+  </property>
+  <property>
+    <name>storm.local.mode.zmq</name>
+    <value>false</value>
+    <description>Whether or not to use ZeroMQ for messaging in local mode. If this is set
+       to false, then Storm will use a pure-Java messaging system. The purpose
+       of this flag is to make it easy to run Storm in local mode by eliminating
+       the need for native dependencies, which can be difficult to install.
+    </description>
+  </property>
+  <property>
+    <name>storm.thrift.transport</name>
+    <value>backtype.storm.security.auth.SimpleTransportPlugin</value>
+    <description>The transport plug-in for Thrift client/server communication.</description>
+  </property>
+  <property>
+    <name>storm.messaging.transport</name>
+    <value>backtype.storm.messaging.netty.Context</value>
+    <description>The transporter for communication among Storm tasks.</description>
+  </property>
+  <property>
+    <name>nimbus.host</name>
+    <value>localhost</value>
+    <description>The host that the master server is running on.</description>
+  </property>
+  <property>
+    <name>nimbus.thrift.port</name>
+    <value>6627</value>
+    <description> Which port the Thrift interface of Nimbus should run on. Clients should
+       connect to this port to upload jars and submit topologies.</description>
+  </property>
+  <property>
+    <name>nimbus.thrift.max_buffer_size</name>
+    <value>1048576</value>
+    <description>The maximum buffer size thrift should use when reading messages.</description>
+  </property>
+  <property>
+    <name>nimbus.childopts</name>
+    <value>-Xmx1024m -Djava.security.auth.login.config=/etc/storm/storm_jaas.conf -javaagent:/usr/lib/storm/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host={0},port=8649,wireformat31x=true,mode=multicast,config=/usr/lib/storm/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Nimbus_JVM</value>
+    <description>This parameter is used by the storm-deploy project to configure the jvm options for the nimbus daemon.</description>
+  </property>
+  <property>
+    <name>nimbus.task.timeout.secs</name>
+    <value>30</value>
+    <description>How long without heartbeating a task can go before nimbus will consider the task dead and reassign it to another location.</description>
+  </property>
+  <property>
+    <name>nimbus.supervisor.timeout.secs</name>
+    <value>60</value>
+    <description>How long before a supervisor can go without heartbeating before nimbus considers it dead and stops assigning new work to it.</description>
+  </property>
+  <property>
+    <name>nimbus.monitor.freq.secs</name>
+    <value>10</value>
+    <description>
+      How often nimbus should wake up to check heartbeats and do reassignments. Note
+       that if a machine ever goes down Nimbus will immediately wake up and take action.
+       This parameter is for checking for failures when there's no explicit event like that occuring.
+    </description>
+  </property>
+  <property>
+    <name>nimbus.cleanup.inbox.freq.secs</name>
+    <value>600</value>
+    <description>How often nimbus should wake the cleanup thread to clean the inbox.</description>
+  </property>
+  <property>
+    <name>nimbus.inbox.jar.expiration.secs</name>
+    <value>3600</value>
+    <description>
+      The length of time a jar file lives in the inbox before being deleted by the cleanup thread.
+
+       Probably keep this value greater than or equal to NIMBUS_CLEANUP_INBOX_JAR_EXPIRATION_SECS.
+       Note that the time it takes to delete an inbox jar file is going to be somewhat more than
+       NIMBUS_CLEANUP_INBOX_JAR_EXPIRATION_SECS (depending on how often NIMBUS_CLEANUP_FREQ_SECS is set to).
+      </description>
+  </property>
+  <property>
+    <name>nimbus.task.launch.secs</name>
+    <value>120</value>
+    <description>A special timeout used when a task is initially launched. During launch, this is the timeout
+       used until the first heartbeat, overriding nimbus.task.timeout.secs.</description>
+  </property>
+  <property>
+    <name>nimbus.reassign</name>
+    <value>true</value>
+    <description>Whether or not nimbus should reassign tasks if it detects that a task goes down.
+       Defaults to true, and it's not recommended to change this value.</description>
+  </property>
+  <property>
+    <name>nimbus.file.copy.expiration.secs</name>
+    <value>600</value>
+    <description>During upload/download with the master, how long an upload or download connection is idle
+       before nimbus considers it dead and drops the connection.</description>
+  </property>
+  <property>
+    <name>nimbus.topology.validator</name>
+    <value>backtype.storm.nimbus.DefaultTopologyValidator</value>
+    <description>A custom class that implements ITopologyValidator that is run whenever a
+       topology is submitted. Can be used to provide business-specific logic for
+       whether topologies are allowed to run or not.</description>
+  </property>
+  <property>
+    <name>ui.port</name>
+    <value>8744</value>
+    <description>Storm UI binds to this port.</description>
+  </property>
+  <property>
+    <name>ui.childopts</name>
+    <value>-Xmx768m -Djava.security.auth.login.config=/etc/storm/storm_jaas.conf</value>
+    <description>Childopts for Storm UI Java process.</description>
+  </property>
+  <property>
+    <name>logviewer.port</name>
+    <value>8000</value>
+    <description>HTTP UI port for log viewer.</description>
+  </property>
+  <property>
+    <name>logviewer.childopts</name>
+    <value>-Xmx128m</value>
+    <description>Childopts for log viewer java process.</description>
+  </property>
+  <property>
+    <name>logviewer.appender.name</name>
+    <value>A1</value>
+    <description>Appender name used by log viewer to determine log directory.</description>
+  </property>
+  <property>
+    <name>drpc.port</name>
+    <value>3772</value>
+    <description>This port is used by Storm DRPC for receiving DPRC requests from clients.</description>
+  </property>
+  <property>
+    <name>drpc.worker.threads</name>
+    <value>64</value>
+    <description>DRPC thrift server worker threads.</description>
+  </property>
+  <property>
+    <name>drpc.queue.size</name>
+    <value>128</value>
+    <description>DRPC thrift server queue size.</description>
+  </property>
+  <property>
+    <name>drpc.invocations.port</name>
+    <value>3773</value>
+    <description>This port on Storm DRPC is used by DRPC topologies to receive function invocations and send results back.</description>
+  </property>
+  <property>
+    <name>drpc.request.timeout.secs</name>
+    <value>600</value>
+    <description>The timeout on DRPC requests within the DRPC server. Defaults to 10 minutes. Note that requests can also
+       timeout based on the socket timeout on the DRPC client, and separately based on the topology message
+       timeout for the topology implementing the DRPC function.</description>
+  </property>
+  <property>
+    <name>drpc.childopts</name>
+    <value>-Xmx768m</value>
+    <description>Childopts for Storm DRPC Java process.</description>
+  </property>
+  <property>
+    <name>transactional.zookeeper.root</name>
+    <value>/transactional</value>
+    <description>The root directory in ZooKeeper for metadata about TransactionalSpouts.</description>
+  </property>
+  <property>
+    <name>transactional.zookeeper.servers</name>
+    <value>null</value>
+    <description>The list of zookeeper servers in which to keep the transactional state. If null (which is default),
+       will use storm.zookeeper.servers</description>
+  </property>
+  <property>
+    <name>transactional.zookeeper.port</name>
+    <value>null</value>
+    <description>The port to use to connect to the transactional zookeeper servers. If null (which is default),
+       will use storm.zookeeper.port</description>
+  </property>
+  <property>
+    <name>supervisor.slots.ports</name>
+    <value>[6700, 6701]</value>
+    <description>A list of ports that can run workers on this supervisor. Each worker uses one port, and
+       the supervisor will only run one worker per port. Use this configuration to tune
+       how many workers run on each machine.</description>
+  </property>
+  <property>
+    <name>supervisor.childopts</name>
+    <value>-Xmx256m -Djava.security.auth.login.config=/etc/storm/storm_jaas.conf -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.port=56431 -javaagent:/usr/lib/storm/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host={0},port=8650,wireformat31x=true,mode=multicast,config=/usr/lib/storm/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Supervisor_JVM</value>
+    <description>This parameter is used by the storm-deploy project to configure the jvm options for the supervisor daemon.</description>
+  </property>
+  <property>
+    <name>supervisor.worker.start.timeout.secs</name>
+    <value>120</value>
+    <description>How long a worker can go without heartbeating during the initial launch before
+       the supervisor tries to restart the worker process. This value override
+       supervisor.worker.timeout.secs during launch because there is additional
+       overhead to starting and configuring the JVM on launch.</description>
+  </property>
+  <property>
+    <name>supervisor.worker.timeout.secs</name>
+    <value>30</value>
+    <description>How long a worker can go without heartbeating before the supervisor tries to restart the worker process.</description>
+  </property>
+  <property>
+    <name>supervisor.monitor.frequency.secs</name>
+    <value>3</value>
+    <description>How often the supervisor checks the worker heartbeats to see if any of them need to be restarted.</description>
+  </property>
+  <property>
+    <name>supervisor.heartbeat.frequency.secs</name>
+    <value>5</value>
+    <description>How often the supervisor sends a heartbeat to the master.</description>
+  </property>
+  <property>
+    <name>supervisor.enable</name>
+    <value>true</value>
+    <description>Whether or not the supervisor should launch workers assigned to it. Defaults
+       to true -- and you should probably never change this value. This configuration
+       is used in the Storm unit tests.</description>
+  </property>
+  <property>
+    <name>worker.childopts</name>
+    <value>-Xmx768m -javaagent:/usr/lib/storm/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host={0},port=8650,wireformat31x=true,mode=multicast,config=/usr/lib/storm/contrib/storm-jmxetric/conf/jmxetric-conf.xml,process=Worker_%ID%_JVM</value>
+    <description>The jvm opts provided to workers launched by this supervisor. All \"%ID%\" substrings are replaced with an identifier for this worker.</description>
+  </property>
+  <property>
+    <name>worker.heartbeat.frequency.secs</name>
+    <value>1</value>
+    <description>How often this worker should heartbeat to the supervisor.</description>
+  </property>
+  <property>
+    <name>task.heartbeat.frequency.secs</name>
+    <value>3</value>
+    <description>How often a task should heartbeat its status to the master.</description>
+  </property>
+  <property>
+    <name>task.refresh.poll.secs</name>
+    <value>10</value>
+    <description>How often a task should sync its connections with other tasks (if a task is
+       reassigned, the other tasks sending messages to it need to refresh their connections).
+       In general though, when a reassignment happens other tasks will be notified
+       almost immediately. This configuration is here just in case that notification doesn't
+       come through.</description>
+  </property>
+  <property>
+    <name>zmq.threads</name>
+    <value>1</value>
+    <description>The number of threads that should be used by the zeromq context in each worker process.</description>
+  </property>
+  <property>
+    <name>zmq.linger.millis</name>
+    <value>5000</value>
+    <description>How long a connection should retry sending messages to a target host when
+       the connection is closed. This is an advanced configuration and can almost
+       certainly be ignored.</description>
+  </property>
+  <property>
+    <name>zmq.hwm</name>
+    <value>0</value>
+    <description>The high water for the ZeroMQ push sockets used for networking. Use this config to prevent buffer explosion
+       on the networking layer.</description>
+  </property>
+  <property>
+    <name>storm.messaging.netty.server_worker_threads</name>
+    <value>1</value>
+    <description>Netty based messaging: The # of worker threads for the server.</description>
+  </property>
+  <property>
+    <name>storm.messaging.netty.client_worker_threads</name>
+    <value>1</value>
+    <description>Netty based messaging: The # of worker threads for the client.</description>
+  </property>
+  <property>
+    <name>storm.messaging.netty.buffer_size</name>
+    <value>5242880</value>
+    <description>Netty based messaging: The buffer size for send/recv buffer.</description>
+  </property>
+  <property>
+    <name>storm.messaging.netty.max_retries</name>
+    <value>30</value>
+    <description>Netty based messaging: The max # of retries that a peer will perform when a remote is not accessible.</description>
+  </property>
+  <property>
+    <name>storm.messaging.netty.max_wait_ms</name>
+    <value>1000</value>
+    <description>Netty based messaging: The max # of milliseconds that a peer will wait.</description>
+  </property>
+  <property>
+    <name>storm.messaging.netty.min_wait_ms</name>
+    <value>100</value>
+    <description>Netty based messaging: The min # of milliseconds that a peer will wait.</description>
+  </property>
+  <property>
+    <name>topology.enable.message.timeouts</name>
+    <value>true</value>
+    <description>True if Storm should timeout messages or not. Defaults to true. This is meant to be used
+       in unit tests to prevent tuples from being accidentally timed out during the test.</description>
+  </property>
+  <property>
+    <name>topology.debug</name>
+    <value>false</value>
+    <description>When set to true, Storm will log every message that's emitted.</description>
+  </property>
+  <property>
+    <name>topology.optimize</name>
+    <value>true</value>
+    <description>Whether or not the master should optimize topologies by running multiple tasks in a single thread where appropriate.</description>
+  </property>
+  <property>
+    <name>topology.workers</name>
+    <value>1</value>
+    <description>How many processes should be spawned around the cluster to execute this
+       topology. Each process will execute some number of tasks as threads within
+       them. This parameter should be used in conjunction with the parallelism hints
+       on each component in the topology to tune the performance of a topology.</description>
+  </property>
+  <property>
+    <name>topology.acker.executors</name>
+    <value>null</value>
+    <description>How many executors to spawn for ackers.
+
+      If this is set to 0, then Storm will immediately ack tuples as soon
+       as they come off the spout, effectively disabling reliability.
+    </description>
+  </property>
+  <property>
+    <name>topology.message.timeout.secs</name>
+    <value>30</value>
+    <description>The maximum amount of time given to the topology to fully process a message
+       emitted by a spout. If the message is not acked within this time frame, Storm
+       will fail the message on the spout. Some spouts implementations will then replay
+       the message at a later time.</description>
+  </property>
+  <property>
+    <name>topology.skip.missing.kryo.registrations</name>
+    <value>false</value>
+    <description> Whether or not Storm should skip the loading of kryo registrations for which it
+       does not know the class or have the serializer implementation. Otherwise, the task will
+       fail to load and will throw an error at runtime. The use case of this is if you want to
+       declare your serializations on the storm.yaml files on the cluster rather than every single
+       time you submit a topology. Different applications may use different serializations and so
+       a single application may not have the code for the other serializers used by other apps.
+       By setting this config to true, Storm will ignore that it doesn't have those other serializations
+       rather than throw an error.</description>
+  </property>
+  <property>
+    <name>topology.max.task.parallelism</name>
+    <value>null</value>
+    <description>The maximum parallelism allowed for a component in this topology. This configuration is
+       typically used in testing to limit the number of threads spawned in local mode.</description>
+  </property>
+  <property>
+    <name>topology.max.spout.pending</name>
+    <value>null</value>
+    <description>The maximum number of tuples that can be pending on a spout task at any given time.
+       This config applies to individual tasks, not to spouts or topologies as a whole.
+
+       A pending tuple is one that has been emitted from a spout but has not been acked or failed yet.
+       Note that this config parameter has no effect for unreliable spouts that don't tag
+       their tuples with a message id.</description>
+  </property>
+  <property>
+    <name>topology.state.synchronization.timeout.secs</name>
+    <value>60</value>
+    <description>The maximum amount of time a component gives a source of state to synchronize before it requests
+       synchronization again.</description>
+  </property>
+  <property>
+    <name>topology.stats.sample.rate</name>
+    <value>0.05</value>
+    <description>The percentage of tuples to sample to produce stats for a task.</description>
+  </property>
+  <property>
+    <name>topology.builtin.metrics.bucket.size.secs</name>
+    <value>60</value>
+    <description>The time period that builtin metrics data in bucketed into.</description>
+  </property>
+  <property>
+    <name>topology.fall.back.on.java.serialization</name>
+    <value>true</value>
+    <description>Whether or not to use Java serialization in a topology.</description>
+  </property>
+  <property>
+    <name>topology.worker.childopts</name>
+    <value>null</value>
+    <description>Topology-specific options for the worker child process. This is used in addition to WORKER_CHILDOPTS.</description>
+  </property>
+  <property>
+    <name>topology.executor.receive.buffer.size</name>
+    <value>1024</value>
+    <description>The size of the Disruptor receive queue for each executor. Must be a power of 2.</description>
+  </property>
+  <property>
+    <name>topology.executor.send.buffer.size</name>
+    <value>1024</value>
+    <description>The size of the Disruptor send queue for each executor. Must be a power of 2.</description>
+  </property>
+  <property>
+    <name>topology.receiver.buffer.size</name>
+    <value>8</value>
+    <description>The maximum number of messages to batch from the thread receiving off the network to the
+       executor queues. Must be a power of 2.</description>
+  </property>
+  <property>
+    <name>topology.transfer.buffer.size</name>
+    <value>1024</value>
+    <description>The size of the Disruptor transfer queue for each worker.</description>
+  </property>
+  <property>
+    <name>topology.tick.tuple.freq.secs</name>
+    <value>null</value>
+    <description>How often a tick tuple from the "__system" component and "__tick" stream should be sent
+       to tasks. Meant to be used as a component-specific configuration.</description>
+  </property>
+  <property>
+    <name>topology.worker.shared.thread.pool.size</name>
+    <value>4</value>
+    <description>The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed
+       via the TopologyContext.</description>
+  </property>
+  <property>
+    <name>topology.disruptor.wait.strategy</name>
+    <value>com.lmax.disruptor.BlockingWaitStrategy</value>
+    <description>Configure the wait strategy used for internal queuing. Can be used to tradeoff latency
+       vs. throughput.</description>
+  </property>
+  <property>
+    <name>topology.executor.send.buffer.size</name>
+    <value>1024</value>
+    <description>The size of the Disruptor send queue for each executor. Must be a power of 2.</description>
+  </property>
+  <property>
+    <name>topology.receiver.buffer.size</name>
+    <value>8</value>
+    <description>The maximum number of messages to batch from the thread receiving off the network to the
+       executor queues. Must be a power of 2.</description>
+  </property>
+  <property>
+    <name>topology.transfer.buffer.size</name>
+    <value>1024</value>
+    <description>The size of the Disruptor transfer queue for each worker.</description>
+  </property>
+  <property>
+    <name>topology.tick.tuple.freq.secs</name>
+    <value>null</value>
+    <description>How often a tick tuple from the "__system" component and "__tick" stream should be sent
+       to tasks. Meant to be used as a component-specific configuration.</description>
+  </property>
+  <property>
+    <name>topology.worker.shared.thread.pool.size</name>
+    <value>4</value>
+    <description>The size of the shared thread pool for worker tasks to make use of. The thread pool can be accessed
+       via the TopologyContext.</description>
+  </property>
+  <property>
+    <name>topology.spout.wait.strategy</name>
+    <value>backtype.storm.spout.SleepSpoutWaitStrategy</value>
+    <description>A class that implements a strategy for what to do when a spout needs to wait. Waiting is
+       triggered in one of two conditions:
+
+       1. nextTuple emits no tuples
+       2. The spout has hit maxSpoutPending and can't emit any more tuples</description>
+  </property>
+  <property>
+    <name>topology.sleep.spout.wait.strategy.time.ms</name>
+    <value>1</value>
+    <description>The amount of milliseconds the SleepEmptyEmitStrategy should sleep for.</description>
+  </property>
+  <property>
+    <name>topology.error.throttle.interval.secs</name>
+    <value>10</value>
+    <description>The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example,
+       an interval of 10 seconds with topology.max.error.report.per.interval set to 5 will only allow 5 errors to be
+       reported to Zookeeper per task for every 10 second interval of time.</description>
+  </property>
+  <property>
+    <name>topology.max.error.report.per.interval</name>
+    <value>5</value>
+    <description>The interval in seconds to use for determining whether to throttle error reported to Zookeeper. For example,
+       an interval of 10 seconds with topology.max.error.report.per.interval set to 5 will only allow 5 errors to be
+       reported to Zookeeper per task for every 10 second interval of time.</description>
+  </property>
+  <property>
+    <name>topology.kryo.factory</name>
+    <value>backtype.storm.serialization.DefaultKryoFactory</value>
+    <description>Class that specifies how to create a Kryo instance for serialization. Storm will then apply
+       topology.kryo.register and topology.kryo.decorators on top of this. The default implementation
+       implements topology.fall.back.on.java.serialization and turns references off.</description>
+  </property>
+  <property>
+    <name>topology.tuple.serializer</name>
+    <value>backtype.storm.serialization.types.ListDelegateSerializer</value>
+    <description>The serializer class for ListDelegate (tuple payload).
+       The default serializer will be ListDelegateSerializer</description>
+  </property>
+  <property>
+    <name>topology.trident.batch.emit.interval.millis</name>
+    <value>500</value>
+    <description>How often a batch can be emitted in a Trident topology.</description>
+  </property>
+  <property>
+    <name>dev.zookeeper.path</name>
+    <value>/tmp/dev-storm-zookeeper</value>
+    <description>The path to use as the zookeeper dir when running a zookeeper server via
+       "storm dev-zookeeper". This zookeeper instance is only intended for development;
+       it is not a production grade zookeeper setup.</description>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/ganglia_metrics.json
----------------------------------------------------------------------
diff --git a/app-packages/storm/ganglia_metrics.json b/app-packages/storm/ganglia_metrics.json
new file mode 100644
index 0000000..861c4fa
--- /dev/null
+++ b/app-packages/storm/ganglia_metrics.json
@@ -0,0 +1,31 @@
+{
+    "Component": {
+        "NIMBUS": {
+            "totalslots": {
+                "metric": "Total Slots",
+                "pointInTime": false,
+                "temporal": true
+            },
+            "totalexecutors": {
+                "metric": "Total Executors",
+                "pointInTime": false,
+                "temporal": true
+            },
+            "topologies": {
+                "metric": "Topologies",
+                "pointInTime": false,
+                "temporal": true
+            },
+            "totaltasks": {
+                "metric": "Total Tasks",
+                "pointInTime": false,
+                "temporal": true
+            },
+            "usedslots": {
+                "metric": "Used Slots",
+                "pointInTime": false,
+                "temporal": true
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/jmx_metrics.json
----------------------------------------------------------------------
diff --git a/app-packages/storm/jmx_metrics.json b/app-packages/storm/jmx_metrics.json
new file mode 100644
index 0000000..f7d4e60
--- /dev/null
+++ b/app-packages/storm/jmx_metrics.json
@@ -0,0 +1,31 @@
+{
+    "Component": {
+        "NIMBUS": {
+            "FreeSlots": {
+                "metric": "$['slots.free']",
+                "pointInTime": true,
+                "temporal": false
+            },
+            "Tasks": {
+                "metric": "$['tasks.total']",
+                "pointInTime": true,
+                "temporal": false
+            },
+            "Executors": {
+                "metric": "$['executors.total']",
+                "pointInTime": true,
+                "temporal": false
+            },
+            "Topologies": {
+                "metric": "$['topologies']",
+                "pointInTime": true,
+                "temporal": false
+            },
+            "NimbusUptime": {
+                "metric": "$['nimbus.uptime']",
+                "pointInTime": true,
+                "temporal": false
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/metainfo.xml
----------------------------------------------------------------------
diff --git a/app-packages/storm/metainfo.xml b/app-packages/storm/metainfo.xml
new file mode 100644
index 0000000..7edd794
--- /dev/null
+++ b/app-packages/storm/metainfo.xml
@@ -0,0 +1,145 @@
+<?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>STORM</name>
+    <comment>Apache Hadoop Stream processing framework</comment>
+    <version>0.9.1.2.1</version>
+
+    <exportGroups>
+      <exportGroup>
+        <name>QuickLinks</name>
+        <exports>
+          <export>
+            <name>org.apache.slider.jmx</name>
+            <value>http://${STORM_REST_API_HOST}:${site.global.rest_api_port}/api/cluster/summary</value>
+          </export>
+          <export>
+            <name>org.apache.slider.monitor</name>
+            <value>http://${STORM_UI_SERVER_HOST}:${site.storm-site.ui.port}</value>
+          </export>
+          <export>
+            <name>org.apache.slider.metrics</name>
+            <value>http://${site.global.ganglia_server_host}/cgi-bin/rrd.py?c=${site.global.ganglia_server_id}</value>
+          </export>
+          <export>
+            <name>org.apache.slider.ganglia</name>
+            <value>http://${site.global.ganglia_server_host}/ganglia?c=${site.global.ganglia_server_id}</value>
+          </export>
+        </exports>
+      </exportGroup>
+    </exportGroups>
+
+    <commandOrders>
+      <commandOrder>
+        <command>NIMBUS-START</command>
+        <requires>SUPERVISOR-INSTALLED,STORM_UI_SERVER-INSTALLED,DRPC_SERVER-INSTALLED,STORM_REST_API-INSTALLED
+        </requires>
+      </commandOrder>
+      <commandOrder>
+        <command>SUPERVISOR-START</command>
+        <requires>NIMBUS-STARTED</requires>
+      </commandOrder>
+      <commandOrder>
+        <command>DRPC_SERVER-START</command>
+        <requires>NIMBUS-STARTED</requires>
+      </commandOrder>
+      <commandOrder>
+        <command>STORM_REST_API-START</command>
+        <requires>NIMBUS-STARTED,DRPC_SERVER-STARTED,STORM_UI_SERVER-STARTED</requires>
+      </commandOrder>
+      <commandOrder>
+        <command>STORM_UI_SERVER-START</command>
+        <requires>NIMBUS-STARTED</requires>
+      </commandOrder>
+    </commandOrders>
+
+    <components>
+
+      <component>
+        <name>NIMBUS</name>
+        <category>MASTER</category>
+        <commandScript>
+          <script>scripts/nimbus.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>STORM_REST_API</name>
+        <category>MASTER</category>
+        <commandScript>
+          <script>scripts/rest_api.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>SUPERVISOR</name>
+        <category>SLAVE</category>
+        <commandScript>
+          <script>scripts/supervisor.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>STORM_UI_SERVER</name>
+        <category>MASTER</category>
+        <publishConfig>true</publishConfig>
+        <commandScript>
+          <script>scripts/ui_server.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+
+      <component>
+        <name>DRPC_SERVER</name>
+        <category>MASTER</category>
+        <commandScript>
+          <script>scripts/drpc_server.py</script>
+          <scriptType>PYTHON</scriptType>
+          <timeout>600</timeout>
+        </commandScript>
+      </component>
+    </components>
+
+    <osSpecifics>
+      <osSpecific>
+        <osType>any</osType>
+        <packages>
+          <package>
+            <type>tarball</type>
+            <name>files/apache-storm-0.9.1.2.1.1.0-237.tar.gz</name>
+          </package>
+        </packages>
+      </osSpecific>
+    </osSpecifics>
+
+    <configuration-dependencies>
+      <config-type>storm-site</config-type>
+      <config-type>global</config-type>
+    </configuration-dependencies>
+  </application>
+</metainfo>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/package/files/apache-storm-0.9.1.2.1.1.0-237.tar.gz.REPLACE
----------------------------------------------------------------------
diff --git a/app-packages/storm/package/files/apache-storm-0.9.1.2.1.1.0-237.tar.gz.REPLACE b/app-packages/storm/package/files/apache-storm-0.9.1.2.1.1.0-237.tar.gz.REPLACE
new file mode 100644
index 0000000..dd934d5
--- /dev/null
+++ b/app-packages/storm/package/files/apache-storm-0.9.1.2.1.1.0-237.tar.gz.REPLACE
@@ -0,0 +1,16 @@
+# 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.
+
+Replace with the actual storm package.

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/package/scripts/drpc_server.py
----------------------------------------------------------------------
diff --git a/app-packages/storm/package/scripts/drpc_server.py b/app-packages/storm/package/scripts/drpc_server.py
new file mode 100644
index 0000000..a01d0f0
--- /dev/null
+++ b/app-packages/storm/package/scripts/drpc_server.py
@@ -0,0 +1,55 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+import sys
+from resource_management import *
+from storm import storm
+from service import service
+
+class DrpcServer(Script):
+  def install(self, env):
+    self.install_packages(env)
+
+  def configure(self, env):
+    import params
+    env.set_params(params)
+
+    storm()
+
+  def start(self, env):
+    import params
+    env.set_params(params)
+    self.configure(env)
+
+    service("drpc", action="start")
+
+  def stop(self, env):
+    import params
+    env.set_params(params)
+
+    service("drpc", action="stop")
+
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+    check_process_status(status_params.pid_drpc)
+
+if __name__ == "__main__":
+  DrpcServer().execute()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/package/scripts/nimbus.py
----------------------------------------------------------------------
diff --git a/app-packages/storm/package/scripts/nimbus.py b/app-packages/storm/package/scripts/nimbus.py
new file mode 100644
index 0000000..c7c3120
--- /dev/null
+++ b/app-packages/storm/package/scripts/nimbus.py
@@ -0,0 +1,55 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+import sys
+from resource_management import *
+from storm import storm
+from service import service
+
+class Nimbus(Script):
+  def install(self, env):
+    self.install_packages(env)
+
+  def configure(self, env):
+    import params
+    env.set_params(params)
+
+    storm()
+
+  def start(self, env):
+    import params
+    env.set_params(params)
+    self.configure(env)
+
+    service("nimbus", action="start")
+
+  def stop(self, env):
+    import params
+    env.set_params(params)
+
+    service("nimbus", action="stop")
+
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+    check_process_status(status_params.pid_nimbus)
+
+if __name__ == "__main__":
+  Nimbus().execute()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/app-packages/storm/package/scripts/params.py b/app-packages/storm/package/scripts/params.py
new file mode 100644
index 0000000..cf21b27
--- /dev/null
+++ b/app-packages/storm/package/scripts/params.py
@@ -0,0 +1,58 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+from resource_management import *
+import status_params
+
+# server configurations
+config = Script.get_config()
+
+app_root = config['configurations']['global']['app_root']
+conf_dir = format("{app_root}/conf")
+storm_user = config['configurations']['global']['app_user']
+log_dir = config['configurations']['global']['app_log_dir']
+pid_dir = status_params.pid_dir
+local_dir = config['configurations']['storm-site']['storm.local.dir']
+user_group = config['configurations']['global']['user_group']
+java64_home = config['hostLevelParams']['java_home']
+nimbus_host = config['configurations']['storm-site']['nimbus.host']
+nimbus_port = config['configurations']['storm-site']['nimbus.thrift.port']
+nimbus_host = config['configurations']['storm-site']['nimbus.host']
+rest_api_port = config['configurations']['global']['rest_api_port']
+rest_api_admin_port = config['configurations']['global']['rest_api_admin_port']
+rest_api_conf_file = format("{conf_dir}/config.yaml")
+rest_lib_dir = format("{app_root}/contrib/storm-rest")
+storm_bin = format("{app_root}/bin/storm")
+
+ganglia_installed = config['configurations']['global']['ganglia_enabled']
+if ganglia_installed:
+  ganglia_report_interval = 60
+  ganglia_server = config['configurations']['global']['ganglia_server_host']
+  ganglia_port = config['configurations']['global']['ganglia_server_port']
+
+_authentication = config['configurations']['core-site']['hadoop.security.authentication']
+security_enabled = ( not is_empty(_authentication) and _authentication == 'kerberos')
+
+if security_enabled:
+  _hostname_lowercase = config['hostname'].lower()
+  _kerberos_domain = config['configurations']['global']['kerberos_domain']
+  _storm_principal_name = config['configurations']['global']['storm_principal_name']
+  storm_jaas_principal = _storm_principal_name.replace('_HOST', _hostname_lowercase)
+  storm_keytab_path = config['configurations']['global']['storm_keytab']

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/package/scripts/rest_api.py
----------------------------------------------------------------------
diff --git a/app-packages/storm/package/scripts/rest_api.py b/app-packages/storm/package/scripts/rest_api.py
new file mode 100644
index 0000000..33d8924
--- /dev/null
+++ b/app-packages/storm/package/scripts/rest_api.py
@@ -0,0 +1,57 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+import sys
+from resource_management import *
+from storm import storm
+from service import service
+
+
+class StormRestApi(Script):
+  def install(self, env):
+    self.install_packages(env)
+    self.configure(env)
+
+  def configure(self, env):
+    import params
+    env.set_params(params)
+
+    storm()
+
+  def start(self, env):
+    import params
+    env.set_params(params)
+    self.configure(env)
+
+    service("rest_api", action="start")
+
+  def stop(self, env):
+    import params
+    env.set_params(params)
+
+    service("rest_api", action="stop")
+
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+    check_process_status(status_params.pid_rest_api)
+
+if __name__ == "__main__":
+  StormRestApi().execute()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/package/scripts/service.py
----------------------------------------------------------------------
diff --git a/app-packages/storm/package/scripts/service.py b/app-packages/storm/package/scripts/service.py
new file mode 100644
index 0000000..10fa5b9
--- /dev/null
+++ b/app-packages/storm/package/scripts/service.py
@@ -0,0 +1,95 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+
+from resource_management import *
+import time
+
+
+def service(
+    name,
+    action='start'):
+  import params
+  import status_params
+
+  pid_file = status_params.pid_files[name]
+  no_op_test = format("ls {pid_file} >/dev/null 2>&1 && ps `cat {pid_file}` >/dev/null 2>&1")
+  jps_path = format("{java64_home}/bin/jps")
+  grep_and_awk = "| grep -v grep | awk '{print $1}'"
+
+  if name == 'ui':
+    #process_cmd = "^java.+backtype.storm.ui.core$"
+    pid_chk_cmd = format("{jps_path} -vl | grep \"^[0-9 ]*backtype.storm.ui.core\" {grep_and_awk}  > {pid_file}")
+  elif name == "rest_api":
+    process_cmd = format("{java64_home}/bin/java -jar {rest_lib_dir}/`ls {rest_lib_dir} | grep -wE storm-rest-[0-9.-]+\.jar` server")
+    crt_pid_cmd = format("pgrep -f \"{process_cmd}\" && pgrep -f \"{process_cmd}\" > {pid_file}")
+  else:
+    #process_cmd = format("^java.+backtype.storm.daemon.{name}$")
+    pid_chk_cmd = format("{jps_path} -vl | grep \"^[0-9 ]*backtype.storm.daemon.{name}\" {grep_and_awk}  > {pid_file}")
+
+  if action == "start":
+    if name == "rest_api":
+      cmd = format("{process_cmd} {rest_api_conf_file} > {log_dir}/restapi.log")
+    else:
+      cmd = format("env JAVA_HOME={java64_home} PATH=$PATH:{java64_home}/bin STORM_BASE_DIR={app_root} STORM_CONF_DIR={conf_dir} {storm_bin} {name}")
+
+    Execute(cmd,
+            not_if=no_op_test,
+            user=params.storm_user,
+            logoutput=False,
+            wait_for_finish=False
+    )
+
+    if name == "rest_api":
+      Execute(crt_pid_cmd,
+              user=params.storm_user,
+              logoutput=True,
+              tries=6,
+              try_sleep=10
+      )
+    else:
+      content = None
+      for i in xrange(12):
+        Execute(pid_chk_cmd,
+                user=params.storm_user,
+                logoutput=True
+        )
+        with open(pid_file) as f:
+          content = f.readline().strip()
+        if content.isdigit():
+          break;
+        File(pid_file, action = "delete")
+        time.sleep(10)
+        pass
+
+      if not content.isdigit():
+        raise Fail(format("Unable to start {name}"))
+
+  elif action == "stop":
+    process_dont_exist = format("! ({no_op_test})")
+    pid = format("`cat {pid_file}` >/dev/null 2>&1")
+    Execute(format("kill {pid}"),
+            not_if=process_dont_exist
+    )
+    Execute(format("kill -9 {pid}"),
+            not_if=format("sleep 2; {process_dont_exist} || sleep 20; {process_dont_exist}"),
+            ignore_failures=True
+    )
+    Execute(format("rm -f {pid_file}"))

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/package/scripts/status_params.py
----------------------------------------------------------------------
diff --git a/app-packages/storm/package/scripts/status_params.py b/app-packages/storm/package/scripts/status_params.py
new file mode 100644
index 0000000..eab83cf
--- /dev/null
+++ b/app-packages/storm/package/scripts/status_params.py
@@ -0,0 +1,36 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+from resource_management import *
+
+config = Script.get_config()
+
+pid_dir = config['configurations']['global']['app_pid_dir']
+pid_nimbus = format("{pid_dir}/nimbus.pid")
+pid_supervisor = format("{pid_dir}/supervisor.pid")
+pid_drpc = format("{pid_dir}/drpc.pid")
+pid_ui = format("{pid_dir}/ui.pid")
+pid_logviewer = format("{pid_dir}/logviewer.pid")
+pid_rest_api = format("{pid_dir}/restapi.pid")
+pid_files = {"logviewer":pid_logviewer,
+             "ui": pid_ui,
+             "nimbus": pid_nimbus,
+             "supervisor": pid_supervisor,
+             "drpc": pid_drpc,
+             "rest_api": pid_rest_api}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/package/scripts/storm.py
----------------------------------------------------------------------
diff --git a/app-packages/storm/package/scripts/storm.py b/app-packages/storm/package/scripts/storm.py
new file mode 100644
index 0000000..bce272b
--- /dev/null
+++ b/app-packages/storm/package/scripts/storm.py
@@ -0,0 +1,50 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+from resource_management import *
+from yaml_config import yaml_config
+import sys
+
+def storm():
+  import params
+
+  Directory([params.log_dir, params.pid_dir, params.local_dir, params.conf_dir],
+            owner=params.storm_user,
+            group=params.user_group,
+            recursive=True
+  )
+
+  File(format("{conf_dir}/config.yaml"),
+            content=Template("config.yaml.j2"),
+            owner = params.storm_user,
+            group = params.user_group
+  )
+
+  yaml_config( "storm.yaml",
+               conf_dir = params.conf_dir,
+               configurations = params.config['configurations']['storm-site'],
+               owner = params.storm_user,
+               group = params.user_group
+  )
+  
+  if params.security_enabled:
+    TemplateConfig( format("{conf_dir}/storm_jaas.conf"),
+      owner = params.storm_user
+    )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/package/scripts/supervisor.py
----------------------------------------------------------------------
diff --git a/app-packages/storm/package/scripts/supervisor.py b/app-packages/storm/package/scripts/supervisor.py
new file mode 100644
index 0000000..47c20c9
--- /dev/null
+++ b/app-packages/storm/package/scripts/supervisor.py
@@ -0,0 +1,61 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+import sys
+from resource_management import *
+from yaml_config import yaml_config
+from storm import storm
+from service import service
+
+
+class Supervisor(Script):
+  def install(self, env):
+    self.install_packages(env)
+
+  def configure(self, env):
+    import params
+    env.set_params(params)
+    storm()
+
+  def start(self, env):
+    import params
+    env.set_params(params)
+    self.configure(env)
+
+    service("supervisor", action="start")
+    service("logviewer", action="start")
+
+  def stop(self, env):
+    import params
+    env.set_params(params)
+
+    service("supervisor", action="stop")
+    service("logviewer", action="stop")
+
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+
+    check_process_status(status_params.pid_supervisor)
+
+
+if __name__ == "__main__":
+  Supervisor().execute()
+

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/package/scripts/ui_server.py
----------------------------------------------------------------------
diff --git a/app-packages/storm/package/scripts/ui_server.py b/app-packages/storm/package/scripts/ui_server.py
new file mode 100644
index 0000000..0fe7cd2
--- /dev/null
+++ b/app-packages/storm/package/scripts/ui_server.py
@@ -0,0 +1,55 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+import sys
+from resource_management import *
+from storm import storm
+from service import service
+
+class UiServer(Script):
+  def install(self, env):
+    self.install_packages(env)
+
+  def configure(self, env):
+    import params
+    env.set_params(params)
+
+    storm()
+
+  def start(self, env):
+    import params
+    env.set_params(params)
+    self.configure(env)
+
+    service("ui", action="start")
+
+  def stop(self, env):
+    import params
+    env.set_params(params)
+
+    service("ui", action="stop")
+
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+    check_process_status(status_params.pid_ui)
+
+if __name__ == "__main__":
+  UiServer().execute()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/package/scripts/yaml_config.py
----------------------------------------------------------------------
diff --git a/app-packages/storm/package/scripts/yaml_config.py b/app-packages/storm/package/scripts/yaml_config.py
new file mode 100644
index 0000000..39261be
--- /dev/null
+++ b/app-packages/storm/package/scripts/yaml_config.py
@@ -0,0 +1,69 @@
+#!/usr/bin/env python
+"""
+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.
+
+"""
+
+import re
+from resource_management import *
+
+def escape_yaml_propetry(value):
+  unquouted = False
+  unquouted_values = ["null","Null","NULL","true","True","TRUE","false","False","FALSE","YES","Yes","yes","NO","No","no","ON","On","on","OFF","Off","off"]
+  
+  if value in unquouted_values:
+    unquouted = True
+
+  # if is list [a,b,c]
+  if re.match('^\w*\[.+\]\w*$', value):
+    unquouted = True
+    
+  try:
+    int(value)
+    unquouted = True
+  except ValueError:
+    pass
+  
+  try:
+    float(value)
+    unquouted = True
+  except ValueError:
+    pass
+  
+  if not unquouted:
+    value = value.replace("'","''")
+    value = "'"+value+"'"
+    
+  return value
+
+def yaml_config(
+  filename,
+  configurations = None,
+  conf_dir = None,
+  mode = None,
+  owner = None,
+  group = None
+):
+    config_content = source.InlineTemplate('''{% for key, value in configurations_dict.items() %}{{ key }}: {{ escape_yaml_propetry(value) }}
+{% endfor %}''', configurations_dict=configurations, extra_imports=[escape_yaml_propetry])
+
+    File (format("{conf_dir}/{filename}"),
+      content = config_content,
+      owner = owner,
+      group = group,
+      mode = mode
+    )

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/package/templates/config.yaml.j2
----------------------------------------------------------------------
diff --git a/app-packages/storm/package/templates/config.yaml.j2 b/app-packages/storm/package/templates/config.yaml.j2
new file mode 100644
index 0000000..32d2c99
--- /dev/null
+++ b/app-packages/storm/package/templates/config.yaml.j2
@@ -0,0 +1,48 @@
+# 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.
+
+nimbusHost: {{nimbus_host}}
+nimbusPort: {{nimbus_port}}
+
+# HTTP-specific options.
+http:
+
+  # The port on which the HTTP server listens for service requests.
+  port: {{rest_api_port}}
+
+  # The port on which the HTTP server listens for administrative requests.
+  adminPort: {{rest_api_admin_port}}
+
+{% if ganglia_installed %}
+enableGanglia: {{ganglia_installed}}
+
+# ganglia configuration (necessary if ganglia reporting is enabled)
+ganglia:
+
+  # how often to report to ganglia metrics (in seconds)
+  reportInterval: {{ganglia_report_interval}}
+
+  # the hostname of the gmond server where storm cluster metrics will be sent
+  host: {{ganglia_server}}
+  port: {{ganglia_port}}
+
+  # address mode
+  # default is MULTICAST
+  addressMode: "UNICAST"
+
+  # an <IP>:<HOSTNAME> pair to spoof
+  # this allows us to simulate storm cluster metrics coming from a specific host
+  #spoof: "192.168.1.1:storm"
+{% endif %}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/package/templates/storm_jaas.conf.j2
----------------------------------------------------------------------
diff --git a/app-packages/storm/package/templates/storm_jaas.conf.j2 b/app-packages/storm/package/templates/storm_jaas.conf.j2
new file mode 100644
index 0000000..4031d22
--- /dev/null
+++ b/app-packages/storm/package/templates/storm_jaas.conf.j2
@@ -0,0 +1,26 @@
+/*
+ * 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
+   keyTab="{{storm_keytab_path}}"
+   storeKey=true
+   useTicketCache=false
+   serviceName="zookeeper"
+   principal="{{storm_jaas_principal}}";
+};

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/32c7ea98/app-packages/storm/resources.json
----------------------------------------------------------------------
diff --git a/app-packages/storm/resources.json b/app-packages/storm/resources.json
new file mode 100644
index 0000000..b184a40
--- /dev/null
+++ b/app-packages/storm/resources.json
@@ -0,0 +1,31 @@
+{
+  "schema" : "http://example.org/specification/v2.0.0",
+  "metadata" : {
+  },
+  "global" : {
+  },
+  "components": {
+    "slider-appmaster": {
+    },
+    "NIMBUS": {
+      "yarn.role.priority": "1",
+      "yarn.component.instances": "1"
+    },
+    "STORM_REST_API": {
+      "yarn.role.priority": "2",
+      "yarn.component.instances": "1"
+    },
+    "STORM_UI_SERVER": {
+      "yarn.role.priority": "3",
+      "yarn.component.instances": "1"
+    },
+    "DRPC_SERVER": {
+      "yarn.role.priority": "4",
+      "yarn.component.instances": "1"
+    },
+    "SUPERVISOR": {
+      "yarn.role.priority": "5",
+      "yarn.component.instances": "1"
+    }
+  }
+}
\ No newline at end of file


[45/50] [abbrv] git commit: SLIDER-225. mvn clean install should succeed without any additional parameter

Posted by sm...@apache.org.
SLIDER-225. mvn clean install should succeed without any additional parameter


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

Branch: refs/heads/master
Commit: bedf1c1d0c2724255c52dd8e150685e2b524d40e
Parents: ab71b65
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Thu Jul 10 18:49:28 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Thu Jul 10 18:49:28 2014 -0700

----------------------------------------------------------------------
 .../slider/providers/hbase/funtest/HBaseCommandTestBase.groovy      | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/bedf1c1d/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/HBaseCommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/HBaseCommandTestBase.groovy b/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/HBaseCommandTestBase.groovy
index fe9370e..8bad590 100644
--- a/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/HBaseCommandTestBase.groovy
+++ b/slider-providers/hbase/hbase-funtests/src/test/groovy/org/apache/slider/providers/hbase/funtest/HBaseCommandTestBase.groovy
@@ -52,6 +52,7 @@ abstract class HBaseCommandTestBase extends CommandTestBase {
 
   @BeforeClass
   public static void extendClasspath() {
+    assumeFunctionalTestsEnabled()
     addExtraJar(HBaseClientProvider)
   }
 


[19/50] [abbrv] git commit: SLIDER-215 Add functional tests for HBase app package

Posted by sm...@apache.org.
SLIDER-215 Add functional tests for HBase app package


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

Branch: refs/heads/master
Commit: 2d8f68d1bea3937a5182e8c6149f3c219f034dfa
Parents: 0d0ebe9
Author: tedyu <yu...@gmail.com>
Authored: Mon Jul 7 15:03:17 2014 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Jul 7 15:03:17 2014 -0700

----------------------------------------------------------------------
 app-packages/hbase/appConfig.json               |   7 +-
 app-packages/hbase/pom.xml                      | 162 ++++++++++++++++++-
 app-packages/hbase/resources.json               |   8 +-
 app-packages/hbase/src/assembly/hbase.xml       |   2 +-
 .../hbase/HBaseAgentCommandTestBase.groovy      |  52 ++++++
 .../slider/funtest/hbase/HBaseBasicIT.groovy    | 114 +++++++++++++
 .../funtest/hbase/HBaseMonitorSSLIT.groovy      |  73 +++++++++
 .../hbase/StubToForceGroovyTestsToCompile.java  |  22 +++
 .../test/resources/appConfig_monitor_ssl.json   |  68 ++++++++
 .../hbase/src/test/resources/resources.json     |  21 +++
 10 files changed, 522 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/2d8f68d1/app-packages/hbase/appConfig.json
----------------------------------------------------------------------
diff --git a/app-packages/hbase/appConfig.json b/app-packages/hbase/appConfig.json
index 81fcf2c..3a810f7 100644
--- a/app-packages/hbase/appConfig.json
+++ b/app-packages/hbase/appConfig.json
@@ -3,8 +3,8 @@
   "metadata": {
   },
   "global": {
-    "agent.conf": "/slider/agent/conf/agent.ini",
-    "application.def": "/slider/${app.package.name}.zip",
+    "agent.conf": "agent.ini",
+    "application.def": "${app.package.name}.zip",
     "create.default.zookeeper.node": "true",
     "config_types": "core-site,hdfs-site,hbase-site",
     "java_home": "/usr/jdk64/jdk1.7.0_45",
@@ -16,8 +16,11 @@
     "site.global.app_install_dir": "${AGENT_WORK_ROOT}/app/install",
     "site.global.hbase_master_heapsize": "1024m",
     "site.global.hbase_regionserver_heapsize": "1024m",
+    "site.global.hbase_instance_name": "instancename",
+    "site.global.hbase_root_password": "secret",
     "site.global.user_group": "hadoop",
     "site.global.security_enabled": "false",
+    "site.global.monitor_protocol": "http",
     "site.global.ganglia_server_host": "${NN_HOST}",
     "site.global.ganglia_server_port": "8667",
     "site.global.ganglia_server_id": "Application1",

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/2d8f68d1/app-packages/hbase/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/hbase/pom.xml b/app-packages/hbase/pom.xml
index 6872ac8..3854496 100644
--- a/app-packages/hbase/pom.xml
+++ b/app-packages/hbase/pom.xml
@@ -24,7 +24,7 @@
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>slider-hbase-app-package</artifactId>
-  <packaging>pom</packaging>
+  <packaging>jar</packaging>
   <name>Slider HBase App Package</name>
   <description>Slider HBase App Package</description>
   <properties>
@@ -70,16 +70,90 @@
                 <configuration>
                   <includeArtifactIds>hbase</includeArtifactIds>
                   <includeTypes>tar.gz</includeTypes>
+                  <excludeTransitive>true</excludeTransitive>
                   <outputDirectory>${project.build.directory}/${work.dir}</outputDirectory>
                 </configuration>
               </execution>
             </executions>
           </plugin>
+
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-failsafe-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>run-integration-tests</id>
+                <goals>
+                  <goal>integration-test</goal>
+                  <goal>verify</goal>
+                </goals>
+              </execution>
+            </executions>
+            <configuration>
+              <systemPropertyVariables>
+                <java.net.preferIPv4Stack>true</java.net.preferIPv4Stack>
+                <java.awt.headless>true</java.awt.headless>
+                <!-- this property must be supplied-->
+                <slider.conf.dir>${slider.conf.dir}</slider.conf.dir>
+                <slider.bin.dir>../../slider-assembly/target/slider-${project.version}-all/slider-${project.version}</slider.bin.dir>
+                <test.app.pkg.dir>target</test.app.pkg.dir>
+                <test.app.pkg.file>${app.package.name}.zip</test.app.pkg.file>
+                <test.app.resource>target/test-config/resources.json</test.app.resource>
+                <test.app.template>target/${app.package.name}/appConfig.json</test.app.template>
+              </systemPropertyVariables>
+            </configuration>
+          </plugin>
         </plugins>
       </build>
     </profile>
   </profiles>
 
+  <build>
+    <!-- resources are filtered for dynamic updates. This gets build info in-->
+    <resources>
+      <resource>
+        <directory>src/test/resources</directory>
+        <filtering>true</filtering>
+        <targetPath>${project.build.directory}/test-config</targetPath>
+      </resource>
+    </resources>
+
+    <plugins>
+      <plugin>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>${maven-compiler-plugin.version}</version>
+        <configuration>
+          <compilerId>groovy-eclipse-compiler</compilerId>
+          <!-- set verbose to be true if you want lots of uninteresting messages -->
+          <!-- <verbose>true</verbose> -->
+          <source>${project.java.src.version}</source>
+          <target>${project.java.src.version}</target>
+        </configuration>
+        <dependencies>
+          <dependency>
+            <groupId>org.codehaus.groovy</groupId>
+            <artifactId>groovy-eclipse-compiler</artifactId>
+            <version>${groovy-eclipse-compiler.version}</version>
+          </dependency>
+          <dependency>
+            <groupId>org.codehaus.groovy</groupId>
+            <artifactId>groovy-eclipse-batch</artifactId>
+            <version>${groovy-eclipse-batch.version}</version>
+          </dependency>
+        </dependencies>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <!-- can't figure out how to get the surefire plugin not to pick up the ITs, so skip it entirely -->
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
   <dependencies>
     <dependency>
       <groupId>org.apache.hbase</groupId>
@@ -88,6 +162,92 @@
       <classifier>bin</classifier>
       <type>tar.gz</type>
     </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+       <groupId>org.apache.hbase</groupId>
+       <artifactId>hbase-client</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-minicluster</artifactId>
+      <scope>test</scope>
+    </dependency>
+    
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-protocol</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-common</artifactId>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-it</artifactId>
+      <classifier>tests</classifier>
+        <exclusions>
+          <exclusion>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-client</artifactId>
+          </exclusion>
+        </exclusions>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-hadoop-compat</artifactId>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-hadoop2-compat</artifactId>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-server</artifactId>
+      <classifier>tests</classifier>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.slider</groupId>
+      <artifactId>slider-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.slider</groupId>
+      <artifactId>slider-funtest</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.groovy</groupId>
+      <artifactId>groovy-all</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/2d8f68d1/app-packages/hbase/resources.json
----------------------------------------------------------------------
diff --git a/app-packages/hbase/resources.json b/app-packages/hbase/resources.json
index 9cc1b47..e0ff26f 100644
--- a/app-packages/hbase/resources.json
+++ b/app-packages/hbase/resources.json
@@ -7,13 +7,15 @@
   "components": {
     "HBASE_MASTER": {
       "yarn.role.priority": "1",
-      "yarn.component.instances": "1"
+      "yarn.component.instances": "1",
+      "yarn.memory": "256"
     },
     "slider-appmaster": {
     },
     "HBASE_REGIONSERVER": {
       "yarn.role.priority": "2",
-      "yarn.component.instances": "1"
+      "yarn.component.instances": "1",
+      "yarn.memory": "256"
     }
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/2d8f68d1/app-packages/hbase/src/assembly/hbase.xml
----------------------------------------------------------------------
diff --git a/app-packages/hbase/src/assembly/hbase.xml b/app-packages/hbase/src/assembly/hbase.xml
index 00b69ff..143c88f 100644
--- a/app-packages/hbase/src/assembly/hbase.xml
+++ b/app-packages/hbase/src/assembly/hbase.xml
@@ -23,7 +23,7 @@
   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>
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/2d8f68d1/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy b/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy
new file mode 100644
index 0000000..0026b62
--- /dev/null
+++ b/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.funtest.hbase
+
+import groovy.util.logging.Slf4j
+import org.apache.slider.funtest.framework.AgentCommandTestBase
+import org.junit.After
+import org.junit.Before
+
+@Slf4j
+abstract class AccumuloAgentCommandTestBase extends AgentCommandTestBase {
+  protected static final int HBASE_LAUNCH_WAIT_TIME
+  protected static final int HBASE_GO_LIVE_TIME = 60000
+
+  // parameters must match those found in the default appConfig.json
+  protected static final String INSTANCE_NAME = "instancename"
+  protected static final String USER = "root"
+  protected static final String PASSWORD = "secret"
+
+  static {
+    HBASE_LAUNCH_WAIT_TIME = getTimeOptionMillis(SLIDER_CONFIG,
+      KEY_HBASE_LAUNCH_TIME,
+      1000 * DEFAULT_HBASE_LAUNCH_TIME_SECONDS)
+  }
+
+  abstract public String getClusterName();
+
+  @Before
+  public void prepareCluster() {
+    setupCluster(getClusterName())
+  }
+
+  @After
+  public void destroyCluster() {
+    cleanup(getClusterName())
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/2d8f68d1/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseBasicIT.groovy
----------------------------------------------------------------------
diff --git a/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseBasicIT.groovy b/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseBasicIT.groovy
new file mode 100644
index 0000000..52e19cd
--- /dev/null
+++ b/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseBasicIT.groovy
@@ -0,0 +1,114 @@
+/*
+ * 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.funtest.hbase
+
+import groovy.util.logging.Slf4j
+import org.apache.slider.api.ClusterDescription
+import org.apache.slider.client.SliderClient
+import org.apache.slider.common.SliderKeys
+import org.apache.slider.core.registry.docstore.PublishedConfiguration
+import org.apache.slider.core.registry.info.ServiceInstanceData
+import org.apache.slider.core.registry.retrieve.RegistryRetriever
+import org.apache.slider.funtest.framework.SliderShell
+import org.apache.slider.server.services.curator.CuratorServiceInstance
+import org.junit.Test
+
+@Slf4j
+class HBaseBasicIT extends HBaseAgentCommandTestBase {
+
+  @Override
+  public String getClusterName() {
+    return "test_hbase_basic"
+  }
+
+  @Test
+  public void testHBaseClusterCreate() throws Throwable {
+
+    describe getDescription()
+
+    def path = buildClusterPath(getClusterName())
+    assert !clusterFS.exists(path)
+
+    SliderShell shell = slider(EXIT_SUCCESS,
+      [
+        ACTION_CREATE, getClusterName(),
+        ARG_IMAGE, agentTarballPath.toString(),
+        ARG_TEMPLATE, APP_TEMPLATE,
+        ARG_RESOURCES, APP_RESOURCE
+      ])
+
+    logShell(shell)
+
+    ensureApplicationIsUp(getClusterName())
+
+    // must match the values in src/test/resources/resources.json
+    Map<String, Integer> roleMap = [
+      "HBASE_MASTER" : 1,
+      "HBASE_REGIONSERVER" : 1
+    ];
+
+    //get a slider client against the cluster
+    SliderClient sliderClient = bondToCluster(SLIDER_CONFIG, getClusterName())
+    ClusterDescription cd = sliderClient.clusterDescription
+    assert getClusterName() == cd.name
+
+    log.info("Connected via Client {}", sliderClient.toString())
+
+    //wait for the role counts to be reached
+    waitForRoleCount(sliderClient, roleMap, HBASE_LAUNCH_WAIT_TIME)
+
+    sleep(HBASE_GO_LIVE_TIME)
+
+    clusterLoadOperations(cd, sliderClient)
+  }
+
+
+  public String getDescription() {
+    return "Create a working HBase cluster $clusterName"
+  }
+
+  public static String getMonitorUrl(SliderClient sliderClient, String clusterName) {
+    CuratorServiceInstance<ServiceInstanceData> instance =
+      sliderClient.getRegistry().queryForInstance(SliderKeys.APP_TYPE, clusterName)
+    ServiceInstanceData serviceInstanceData = instance.payload
+    RegistryRetriever retriever = new RegistryRetriever(serviceInstanceData)
+    PublishedConfiguration configuration = retriever.retrieveConfiguration(
+      retriever.getConfigurations(true), "quicklinks", true)
+
+    // must match name set in metainfo.xml
+    String monitorUrl = configuration.entries.get("org.apache.slider.monitor")
+
+    assertNotNull monitorUrl
+    return monitorUrl
+  }
+
+  public static void checkMonitorPage(String monitorUrl) {
+    String monitor = fetchWebPageWithoutError(monitorUrl);
+    assume monitor != null, "Monitor page null"
+    assume monitor.length() > 100, "Monitor page too short"
+    assume monitor.contains("Table Name"), "Monitor page didn't contain expected text"
+  }
+
+  /**
+   * Override point for any cluster load operations
+   */
+  public void clusterLoadOperations(ClusterDescription cd, SliderClient sliderClient) {
+    String monitorUrl = getMonitorUrl(sliderClient, getClusterName())
+    assert monitorUrl.startsWith("http://"), "Monitor URL didn't have expected protocol"
+    checkMonitorPage(monitorUrl)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/2d8f68d1/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseMonitorSSLIT.groovy
----------------------------------------------------------------------
diff --git a/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseMonitorSSLIT.groovy b/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseMonitorSSLIT.groovy
new file mode 100644
index 0000000..12bf7ea
--- /dev/null
+++ b/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseMonitorSSLIT.groovy
@@ -0,0 +1,73 @@
+/*
+ * 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.funtest.hbase
+
+import groovy.util.logging.Slf4j
+import org.apache.slider.api.ClusterDescription
+import org.apache.slider.client.SliderClient
+
+import javax.net.ssl.KeyManager
+import javax.net.ssl.SSLContext
+import javax.net.ssl.TrustManager
+import javax.net.ssl.X509TrustManager
+import java.security.SecureRandom
+import java.security.cert.CertificateException
+import java.security.cert.X509Certificate
+
+@Slf4j
+class HBaseMonitorSSLIT extends HBaseBasicIT {
+  HBaseMonitorSSLIT() {
+    APP_TEMPLATE = "target/test-config/appConfig_monitor_ssl.json"
+  }
+
+  @Override
+  public String getClusterName() {
+    return "test_monitor_ssl";
+  }
+
+  @Override
+  public String getDescription() {
+    return "Test enable monitor SSL $clusterName"
+  }
+
+  @Override
+  public void clusterLoadOperations(ClusterDescription cd, SliderClient sliderClient) {
+    String monitorUrl = getMonitorUrl(sliderClient, getClusterName())
+    assert monitorUrl.startsWith("https://"), "Monitor URL didn't have expected protocol"
+
+    SSLContext ctx = SSLContext.getInstance("SSL");
+    TrustManager[] t = new TrustManager[1];
+    t[0] = new DefaultTrustManager();
+    ctx.init(new KeyManager[0], t, new SecureRandom());
+    SSLContext.setDefault(ctx);
+    checkMonitorPage(monitorUrl)
+  }
+
+  private static class DefaultTrustManager implements X509TrustManager {
+    @Override
+    public void checkClientTrusted(X509Certificate[] arg0, String arg1) throws CertificateException {}
+
+    @Override
+    public void checkServerTrusted(X509Certificate[] arg0, String arg1) throws CertificateException {}
+
+    @Override
+    public X509Certificate[] getAcceptedIssuers() {
+      return null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/2d8f68d1/app-packages/hbase/src/test/java/org/apache/slider/funtest/hbase/StubToForceGroovyTestsToCompile.java
----------------------------------------------------------------------
diff --git a/app-packages/hbase/src/test/java/org/apache/slider/funtest/hbase/StubToForceGroovyTestsToCompile.java b/app-packages/hbase/src/test/java/org/apache/slider/funtest/hbase/StubToForceGroovyTestsToCompile.java
new file mode 100644
index 0000000..8143fcf
--- /dev/null
+++ b/app-packages/hbase/src/test/java/org/apache/slider/funtest/hbase/StubToForceGroovyTestsToCompile.java
@@ -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.
+ */
+
+package org.apache.slider.funtest.hbase;
+
+public class StubToForceGroovyTestsToCompile {
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/2d8f68d1/app-packages/hbase/src/test/resources/appConfig_monitor_ssl.json
----------------------------------------------------------------------
diff --git a/app-packages/hbase/src/test/resources/appConfig_monitor_ssl.json b/app-packages/hbase/src/test/resources/appConfig_monitor_ssl.json
new file mode 100644
index 0000000..37d72d0
--- /dev/null
+++ b/app-packages/hbase/src/test/resources/appConfig_monitor_ssl.json
@@ -0,0 +1,68 @@
+{
+  "schema": "http://example.org/specification/v2.0.0",
+  "metadata": {
+  },
+  "global": {
+    "agent.conf": "agent.ini",
+    "application.def": "${app.package.name}.zip",
+    "create.default.zookeeper.node": "true",
+    "config_types": "core-site,hdfs-site,hbase-site",
+    "java_home": "/usr/jdk64/jdk1.7.0_45",
+    "package_list": "files/hbase-${hbase.version}-bin.tar.gz",
+    "site.global.app_user": "yarn",
+    "site.global.app_log_dir": "app/log",
+    "site.global.app_pid_dir": "${AGENT_WORK_ROOT}/app/run",
+    "site.global.app_root": "${AGENT_WORK_ROOT}/app/install/hbase-${hbase.version}",
+    "site.global.app_install_dir": "${AGENT_WORK_ROOT}/app/install",
+    "site.global.hbase_master_heapsize": "1024m",
+    "site.global.hbase_regionserver_heapsize": "1024m",
+    "site.global.hbase_instance_name": "instancename",
+    "site.global.hbase_root_password": "secret",
+    "site.global.user_group": "hadoop",
+    "site.global.security_enabled": "false",
+    "site.global.monitor_protocol": "https",
+    "site.global.ganglia_server_host": "${NN_HOST}",
+    "site.global.ganglia_server_port": "8667",
+    "site.global.ganglia_server_id": "Application1",
+    "site.hbase-site.hbase.hstore.flush.retries.number": "120",
+    "site.hbase-site.hbase.client.keyvalue.maxsize": "10485760",
+    "site.hbase-site.hbase.hstore.compactionThreshold": "3",
+    "site.hbase-site.hbase.rootdir": "${DEFAULT_DATA_DIR}/data",
+    "site.hbase-site.hbase.stagingdir": "${DEFAULT_DATA_DIR}/staging",
+    "site.hbase-site.hbase.regionserver.handler.count": "60",
+    "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.superuser": "yarn",
+    "site.hbase-site.hbase.zookeeper.property.clientPort": "2181",
+    "site.hbase-site.hbase.regionserver.global.memstore.upperLimit": "0.4",
+    "site.hbase-site.zookeeper.session.timeout": "30000",
+    "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.hregion.max.filesize": "10737418240",
+    "site.hbase-site.hfile.block.cache.size": "0.40",
+    "site.hbase-site.hbase.security.authentication": "simple",
+    "site.hbase-site.hbase.defaults.for.version.skip": "true",
+    "site.hbase-site.hbase.zookeeper.quorum": "${ZK_HOST}",
+    "site.hbase-site.zookeeper.znode.parent": "/hbase-unsecure",
+    "site.hbase-site.hbase.hstore.blockingStoreFiles": "10",
+    "site.hbase-site.hbase.hregion.majorcompaction": "86400000",
+    "site.hbase-site.hbase.security.authorization": "false",
+    "site.hbase-site.hbase.cluster.distributed": "true",
+    "site.hbase-site.hbase.hregion.memstore.mslab.enabled": "true",
+    "site.hbase-site.hbase.client.scanner.caching": "100",
+    "site.hbase-site.hbase.zookeeper.useMulti": "true",
+    "site.hbase-site.hbase.regionserver.info.port": "0",
+    "site.hbase-site.hbase.master.info.port": "${HBASE_MASTER.ALLOCATED_PORT}",
+    "site.hbase-site.hbase.regionserver.port": "0"
+  },
+  "components": {
+    "HBASE_MASTER": {
+    },
+    "slider-appmaster": {
+      "jvm.heapsize": "256M"
+    },
+    "HBASE_REGIONSERVER": {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/2d8f68d1/app-packages/hbase/src/test/resources/resources.json
----------------------------------------------------------------------
diff --git a/app-packages/hbase/src/test/resources/resources.json b/app-packages/hbase/src/test/resources/resources.json
new file mode 100644
index 0000000..e0ff26f
--- /dev/null
+++ b/app-packages/hbase/src/test/resources/resources.json
@@ -0,0 +1,21 @@
+{
+  "schema": "http://example.org/specification/v2.0.0",
+  "metadata": {
+  },
+  "global": {
+  },
+  "components": {
+    "HBASE_MASTER": {
+      "yarn.role.priority": "1",
+      "yarn.component.instances": "1",
+      "yarn.memory": "256"
+    },
+    "slider-appmaster": {
+    },
+    "HBASE_REGIONSERVER": {
+      "yarn.role.priority": "2",
+      "yarn.component.instances": "1",
+      "yarn.memory": "256"
+    }
+  }
+}


[20/50] [abbrv] git commit: SLIDER-215 Addendum fixes typo

Posted by sm...@apache.org.
SLIDER-215 Addendum fixes typo


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

Branch: refs/heads/master
Commit: da59d8812a905948ebe3fb64107a63d9441c902d
Parents: 2d8f68d
Author: tedyu <yu...@gmail.com>
Authored: Mon Jul 7 15:40:14 2014 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Mon Jul 7 15:40:14 2014 -0700

----------------------------------------------------------------------
 .../apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/da59d881/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy b/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy
index 0026b62..fa2ac49 100644
--- a/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy
+++ b/app-packages/hbase/src/test/groovy/org/apache/slider/funtest/hbase/HBaseAgentCommandTestBase.groovy
@@ -23,7 +23,7 @@ import org.junit.After
 import org.junit.Before
 
 @Slf4j
-abstract class AccumuloAgentCommandTestBase extends AgentCommandTestBase {
+abstract class HBaseAgentCommandTestBase extends AgentCommandTestBase {
   protected static final int HBASE_LAUNCH_WAIT_TIME
   protected static final int HBASE_GO_LIVE_TIME = 60000
 


[12/50] [abbrv] git commit: YARN-913 some code reviews and while copying some of this code over to hadoop-trunk branch

Posted by sm...@apache.org.
YARN-913 some code reviews and while copying some of this code over to hadoop-trunk branch


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

Branch: refs/heads/master
Commit: 0b339b7fc76ae95bc2417cb4626d07cb5ce026f2
Parents: 0f9344b
Author: Steve Loughran <st...@apache.org>
Authored: Mon Jul 7 13:57:17 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Jul 7 13:57:17 2014 +0100

----------------------------------------------------------------------
 pom.xml                                         |   2 +-
 .../apache/slider/common/SliderXmlConfKeys.java |   2 +
 .../slider/core/main/ServiceLauncher.java       | 109 +++++++++++++++----
 .../server/appmaster/SliderAppMaster.java       |  46 ++++++--
 .../slider/server/appmaster/state/AppState.java |   2 +-
 .../server/appmaster/web/SliderAMWebApp.java    |   2 +-
 .../server/appmaster/web/SliderAmIpFilter.java  |   1 +
 .../server/services/curator/CuratorService.java |   2 +-
 .../resources/webapps/static/yarn.dt.plugins.js |   4 +-
 9 files changed, 128 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0b339b7f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0076561..c6980be 100644
--- a/pom.xml
+++ b/pom.xml
@@ -132,7 +132,7 @@
     <commons-compress.version>1.4.1</commons-compress.version>
     <commons-logging.version>1.1.3</commons-logging.version>
     <commons-io.version>2.4</commons-io.version>
-    <curator.version>2.4.1</curator.version>
+    <curator.version>2.5.0</curator.version>
     <easymock.version>3.1</easymock.version>
     <guava.version>11.0.2</guava.version>
     <gson.version>2.2.2</gson.version>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0b339b7f/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java b/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
index c7b8ea5..3f16f25 100644
--- a/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/common/SliderXmlConfKeys.java
@@ -153,4 +153,6 @@ public interface SliderXmlConfKeys {
 
   String IPC_CLIENT_FALLBACK_TO_SIMPLE_AUTH =
       "ipc.client.fallback-to-simple-auth-allowed";
+  String HADOOP_HTTP_FILTER_INITIALIZERS =
+      "hadoop.http.filter.initializers";
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0b339b7f/slider-core/src/main/java/org/apache/slider/core/main/ServiceLauncher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/main/ServiceLauncher.java b/slider-core/src/main/java/org/apache/slider/core/main/ServiceLauncher.java
index e22ff54..c92dfda 100644
--- a/slider-core/src/main/java/org/apache/slider/core/main/ServiceLauncher.java
+++ b/slider-core/src/main/java/org/apache/slider/core/main/ServiceLauncher.java
@@ -65,7 +65,8 @@ import java.util.concurrent.atomic.AtomicBoolean;
  */
 @SuppressWarnings("UseOfSystemOutOrSystemErr")
 public class ServiceLauncher<S extends Service>
-  implements LauncherExitCodes, IrqHandler.Interrupted {
+  implements LauncherExitCodes, IrqHandler.Interrupted,
+    Thread.UncaughtExceptionHandler {
   private static final Logger LOG = LoggerFactory.getLogger(
       ServiceLauncher.class);
 
@@ -144,8 +145,8 @@ public class ServiceLauncher<S extends Service>
    * @param conf configuration
    * @param processedArgs arguments after the configuration parameters
    * have been stripped out.
-   * @param addShutdownHook should a shutdown hook be added to terminate
-   * this service on shutdown. Tests should set this to false.
+   * @param addProcessHooks should process failure handlers be added to
+   * terminate this service on shutdown. Tests should set this to false.
    * @throws ClassNotFoundException classname not on the classpath
    * @throws IllegalAccessException not allowed at the class
    * @throws InstantiationException not allowed to instantiate it
@@ -154,13 +155,13 @@ public class ServiceLauncher<S extends Service>
    */
   public int launchService(Configuration conf,
       String[] processedArgs,
-      boolean addShutdownHook)
+      boolean addProcessHooks)
     throws Throwable {
 
     instantiateService(conf);
 
-    // and the shutdown hook if requested
-    if (addShutdownHook) {
+    // add any process shutdown hooks
+    if (addProcessHooks) {
       ServiceShutdownHook shutdownHook = new ServiceShutdownHook(service);
       ShutdownHookManager.get().addShutdownHook(shutdownHook, PRIORITY);
     }
@@ -279,6 +280,53 @@ public class ServiceLauncher<S extends Service>
   }
 
   /**
+   * Uncaught exception handler.
+   * If an error is raised: shutdown
+   * The state of the system is unknown at this point -attempting
+   * a clean shutdown is dangerous. Instead: exit
+   * @param thread thread that failed
+   * @param exception exception
+   */
+  @Override
+  public void uncaughtException(Thread thread, Throwable exception) {
+    if (ShutdownHookManager.get().isShutdownInProgress()) {
+      LOG.error("Thread {} threw an error during shutdown: {}.",
+          thread.toString(),
+          exception,
+          exception);
+    } else if (exception instanceof Error) {
+      try {
+        LOG.error("Thread {} threw an error: {}. Shutting down",
+            thread.toString(),
+            exception,
+            exception);
+      } catch (Throwable err) {
+        // We don't want to not exit because of an issue with logging
+      }
+      if (exception instanceof OutOfMemoryError) {
+        // After catching an OOM java says it is undefined behavior, so don't
+        // even try to clean up or we can get stuck on shutdown.
+        try {
+          System.err.println("Halting due to Out Of Memory Error...");
+        } catch (Throwable err) {
+          // Again we don't want to exit because of logging issues.
+        }
+        ExitUtil.halt(EXIT_EXCEPTION_THROWN);
+      } else {
+        // error other than OutOfMemory
+        exit(convertToExitException(exception));
+      }
+    } else {
+      // simple exception in a thread. There's a policy decision here:
+      // terminate the service vs. keep going after a thread has failed
+      LOG.error("Thread {} threw an exception: {}",
+          thread.toString(),
+          exception,
+          exception);
+    }
+  }
+
+  /**
    * Print a warning: currently this goes to stderr
    * @param text
    */
@@ -286,7 +334,6 @@ public class ServiceLauncher<S extends Service>
     System.err.println(text);
   }
 
-
   /**
    * Report an error. The message is printed to stderr; the exception
    * is logged via the current logger.
@@ -448,25 +495,39 @@ public class ServiceLauncher<S extends Service>
     } catch (ExitUtil.ExitException ee) {
       exitException = ee;
     } catch (Throwable thrown) {
-      int exitCode;
-      String message = thrown.getMessage();
-      if (message == null) {
-        message = thrown.toString();
-      }
-      if (thrown instanceof ExitCodeProvider) {
-        exitCode = ((ExitCodeProvider) thrown).getExitCode();
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("While running {}: {}", getServiceName(), message, thrown);
-        }
-        LOG.error(message);
-      } else {
-        // not any of the service launcher exceptions -assume something worse
-        error(message, thrown);
-        exitCode = EXIT_EXCEPTION_THROWN;
+      exitException = convertToExitException(thrown);
+    }
+    return exitException;
+  }
+
+  /**
+   * Convert the exception to one that can be handed off to ExitUtils;
+   * if it is of the write type it is passed throw as is. If not, a 
+   * new exception with the exit code {@link #EXIT_EXCEPTION_THROWN}
+   * is created, with the argument <code>thrown</code> as the inner cause
+   * @param thrown the exception thrown
+   * @return an exception to terminate the process with
+   */
+  protected ExitUtil.ExitException convertToExitException(Throwable thrown) {
+    ExitUtil.ExitException exitException;
+    int exitCode;
+    String message = thrown.getMessage();
+    if (message == null) {
+      message = thrown.toString();
+    }
+    if (thrown instanceof ExitCodeProvider) {
+      exitCode = ((ExitCodeProvider) thrown).getExitCode();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("While running {}: {}", getServiceName(), message, thrown);
       }
-      exitException = new ExitUtil.ExitException(exitCode, message);
-      exitException.initCause(thrown);
+      LOG.error(message);
+    } else {
+      // not any of the service launcher exceptions -assume something worse
+      error(message, thrown);
+      exitCode = EXIT_EXCEPTION_THROWN;
     }
+    exitException = new ExitUtil.ExitException(exitCode, message);
+    exitException.initCause(thrown);
     return exitException;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0b339b7f/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index 0142028..739d22c 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -179,14 +179,18 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   private YarnRPC yarnRPC;
 
   /** Handle to communicate with the Resource Manager*/
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
   private AMRMClientAsync asyncRMClient;
 
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+
   private RMOperationHandler rmOperationHandler;
 
   /** Handle to communicate with the Node Manager*/
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
   public NMClientAsync nmClientAsync;
 
-  YarnConfiguration conf;
+//  YarnConfiguration conf;
   /**
    * token blob
    */
@@ -197,13 +201,15 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   /**
    * Secret manager
    */
-  ClientToAMTokenSecretManager secretManager;
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+  private ClientToAMTokenSecretManager secretManager;
   
   /** Hostname of the container*/
   private String appMasterHostname = "";
   /* Port on which the app master listens for status updates from clients*/
   private int appMasterRpcPort = 0;
   /** Tracking url to which app master publishes info for clients to monitor*/
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
   private String appMasterTrackingUrl = "";
 
   /** Application Attempt Id ( combination of attemptId and fail count )*/
@@ -235,6 +241,10 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   private final ReentrantLock AMExecutionStateLock = new ReentrantLock();
   private final Condition isAMCompleted = AMExecutionStateLock.newCondition();
 
+  /**
+   * Exit code for the AM to return
+   */
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
   private int amExitCode =  0;
   
   /**
@@ -247,22 +257,30 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   /**
    * Flag to set if the process exit code was set before shutdown started
    */
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
   private boolean spawnedProcessExitedBeforeShutdownTriggered;
 
 
   /** Arguments passed in : raw*/
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
   private SliderAMArgs serviceArgs;
 
   /**
    * ID of the AM container
    */
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
   private ContainerId appMasterContainerID;
 
   /**
    * ProviderService of this cluster
    */
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
   private ProviderService providerService;
 
+  /**
+   * The registry service
+   */
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
   private SliderRegistryService registry;
   
   /**
@@ -277,13 +295,16 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   private int containerMaxMemory;
   private String amCompletionReason;
 
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
   private RoleLaunchService launchService;
   
   //username -null if it is not known/not to be set
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
   private String hadoop_user_name;
   private String service_user_name;
   
   private SliderAMWebApp webApp;
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
   private InetSocketAddress rpcServiceAddress;
   private ProviderService sliderAMProvider;
 
@@ -327,7 +348,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
                SliderUtils.getKerberosRealm());
       UserGroupInformation.setConfiguration(conf);
       UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-      log.debug("Authenticating as " + ugi.toString());
+      log.debug("Authenticating as {}", ugi);
       SliderUtils.verifyPrincipalSet(conf,
           DFSConfigKeys.DFS_NAMENODE_USER_NAME_KEY);
       // always enforce protocol to be token-based.
@@ -356,11 +377,11 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   @Override // RunService
   public Configuration bindArgs(Configuration config, String... args) throws
                                                                       Exception {
-    config = super.bindArgs(config, args);
+    YarnConfiguration yarnConfiguration = new YarnConfiguration(
+        super.bindArgs(config, args));
     serviceArgs = new SliderAMArgs(args);
     serviceArgs.parse();
     //yarn-ify
-    YarnConfiguration yarnConfiguration = new YarnConfiguration(config);
     return SliderUtils.patchConfiguration(yarnConfiguration);
   }
 
@@ -450,11 +471,10 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
     Configuration serviceConf = getConfig();
     // Try to get the proper filtering of static resources through the yarn proxy working
-    serviceConf.set("hadoop.http.filter.initializers",
+    serviceConf.set(HADOOP_HTTP_FILTER_INITIALIZERS,
                     SliderAmFilterInitializer.NAME);
     serviceConf.set(SliderAmIpFilter.WS_CONTEXT_ROOT, WS_CONTEXT_ROOT);
     
-    conf = new YarnConfiguration(serviceConf);
     //get our provider
     MapOperations globalInternalOptions =
       instanceDefinition.getInternalOperations().getGlobalOptions();
@@ -471,9 +491,9 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     sliderAMProvider = new SliderAMProviderService();
     initAndAddService(sliderAMProvider);
     
-    InetSocketAddress address = SliderUtils.getRmSchedulerAddress(conf);
+    InetSocketAddress address = SliderUtils.getRmSchedulerAddress(serviceConf);
     log.info("RM is at {}", address);
-    yarnRPC = YarnRPC.create(conf);
+    yarnRPC = YarnRPC.create(serviceConf);
 
     /*
      * Extract the container ID. This is then
@@ -580,7 +600,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       WebAppService<SliderAMWebApp> webAppService =
         new WebAppService<>("slider", webApp);
 
-      webAppService.init(conf);
+      webAppService.init(serviceConf);
       webAppService.start();
       addService(webAppService);
 
@@ -613,7 +633,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
         applicationACLs = response.getApplicationACLs();
 
         //tell the server what the ACLs are 
-        rpcService.getServer().refreshServiceAcl(conf, new SliderAMPolicyProvider());
+        rpcService.getServer().refreshServiceAcl(serviceConf,
+            new SliderAMPolicyProvider());
       }
 
       // extract container list
@@ -979,7 +1000,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
       // non complete containers should not be here
       assert (status.getState() == ContainerState.COMPLETE);
-      AppState.NodeCompletionResult result = appState.onCompletedNode(conf, status);
+      AppState.NodeCompletionResult result = appState.onCompletedNode(
+          getConfig(), status);
       if (result.containerFailed) {
         RoleInstance ri = result.roleInstance;
         log.error("Role instance {} failed ", ri);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0b339b7f/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
index 9981f68..cc238ff 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/state/AppState.java
@@ -1171,7 +1171,7 @@ public class AppState {
    * @param status the node that has just completed
    * @return NodeCompletionResult
    */
-  public synchronized NodeCompletionResult onCompletedNode(YarnConfiguration amConf,
+  public synchronized NodeCompletionResult onCompletedNode(Configuration amConf,
       ContainerStatus status) {
     ContainerId containerId = status.getContainerId();
     NodeCompletionResult result = new NodeCompletionResult();

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0b339b7f/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
index 0e51f88..4f290af 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAMWebApp.java
@@ -50,7 +50,7 @@ public class SliderAMWebApp extends WebApp {
   public final SliderRegistryService registry;
 
   public SliderAMWebApp(SliderRegistryService registry) {
-    Preconditions.checkNotNull(registry);
+    Preconditions.checkArgument(registry != null, "registry null");
     this.registry = registry;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0b339b7f/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAmIpFilter.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAmIpFilter.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAmIpFilter.java
index fc96284..aba344e 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAmIpFilter.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAmIpFilter.java
@@ -49,6 +49,7 @@ public class SliderAmIpFilter implements Filter {
   private static final long updateInterval = 5 * 60 * 1000;
   public static final String WS_CONTEXT_ROOT = "slider.rest.context.root";
 
+  @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
   private String proxyHost;
   private Set<String> proxyAddresses = null;
   private long lastUpdate;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0b339b7f/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java b/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java
index 657fa57..645bc8f 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/curator/CuratorService.java
@@ -42,7 +42,7 @@ public class CuratorService extends AbstractService {
                         CuratorFramework curator,
                         String basePath) {
     super(name);
-    this.curator = Preconditions.checkNotNull(curator, "null client");
+    this.curator = Preconditions.checkNotNull(curator, "null curator");
     this.basePath = basePath;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0b339b7f/slider-core/src/main/resources/webapps/static/yarn.dt.plugins.js
----------------------------------------------------------------------
diff --git a/slider-core/src/main/resources/webapps/static/yarn.dt.plugins.js b/slider-core/src/main/resources/webapps/static/yarn.dt.plugins.js
index d0bde29..6b8d16c 100644
--- a/slider-core/src/main/resources/webapps/static/yarn.dt.plugins.js
+++ b/slider-core/src/main/resources/webapps/static/yarn.dt.plugins.js
@@ -22,7 +22,7 @@ if (!jQuery.fn.dataTableExt.fnVersionCheck("1.7.5")) {
 // don't filter on hidden html elements for an sType of title-numeric
 $.fn.dataTableExt.ofnSearch['title-numeric'] = function ( sData ) {
    return sData.replace(/\n/g," ").replace( /<.*?>/g, "" );
-}
+};
 
 // 'title-numeric' sort type
 jQuery.fn.dataTableExt.oSort['title-numeric-asc']  = function(a,b) {
@@ -71,7 +71,7 @@ jQuery.fn.dataTableExt.oApi.fnSetFilteringDelay = function ( oSettings, iDelay )
     return this;
   } );
   return this;
-}
+};
 
 function renderHadoopDate(data, type, full) {
   if (type === 'display' || type === 'filter') {


[25/50] [abbrv] git commit: SLIDER-166. agent.ini should be supported by default (Gour Saha via smohanty)

Posted by sm...@apache.org.
SLIDER-166. agent.ini should be supported by default (Gour Saha via smohanty)


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

Branch: refs/heads/master
Commit: adc7f4f13376170ce9d97c070a930697942f0ef8
Parents: c305ce9
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Tue Jul 8 10:16:31 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Tue Jul 8 10:16:31 2014 -0700

----------------------------------------------------------------------
 app-packages/accumulo/appConfig.json                |  1 -
 .../command-logger/slider-pkg/appConfig.json        |  1 -
 app-packages/hbase/appConfig.json                   |  1 -
 app-packages/storm-v0_91/appConfig.json             |  1 -
 .../slider/providers/agent/AgentClientProvider.java | 16 +++++-----------
 .../apache/slider/providers/agent/AgentKeys.java    |  4 ----
 .../providers/agent/AgentProviderService.java       | 12 +++++++-----
 7 files changed, 12 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/adc7f4f1/app-packages/accumulo/appConfig.json
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/appConfig.json b/app-packages/accumulo/appConfig.json
index 6a22336..8fe9a18 100644
--- a/app-packages/accumulo/appConfig.json
+++ b/app-packages/accumulo/appConfig.json
@@ -3,7 +3,6 @@
   "metadata": {
   },
   "global": {
-    "agent.conf": "agent.ini",
     "application.def": "${app.package.name}.zip",
     "config_types": "accumulo-site",
     "java_home": "/usr/jdk64/jdk1.7.0_45",

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/adc7f4f1/app-packages/command-logger/slider-pkg/appConfig.json
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/appConfig.json b/app-packages/command-logger/slider-pkg/appConfig.json
index e7f9700..1d92c59 100644
--- a/app-packages/command-logger/slider-pkg/appConfig.json
+++ b/app-packages/command-logger/slider-pkg/appConfig.json
@@ -3,7 +3,6 @@
     "metadata": {
     },
     "global": {
-        "agent.conf": "agent.ini",
         "application.def": "apache-slider-command-logger.zip",
         "config_types": "cl-site",
         "java_home": "/usr/jdk64/jdk1.7.0_45",

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/adc7f4f1/app-packages/hbase/appConfig.json
----------------------------------------------------------------------
diff --git a/app-packages/hbase/appConfig.json b/app-packages/hbase/appConfig.json
index 07187ec..20cd436 100644
--- a/app-packages/hbase/appConfig.json
+++ b/app-packages/hbase/appConfig.json
@@ -3,7 +3,6 @@
   "metadata": {
   },
   "global": {
-    "agent.conf": "agent.ini",
     "application.def": "${app.package.name}.zip",
     "create.default.zookeeper.node": "true",
     "config_types": "core-site,hdfs-site,hbase-site",

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/adc7f4f1/app-packages/storm-v0_91/appConfig.json
----------------------------------------------------------------------
diff --git a/app-packages/storm-v0_91/appConfig.json b/app-packages/storm-v0_91/appConfig.json
index d6a6679..7ccf412 100644
--- a/app-packages/storm-v0_91/appConfig.json
+++ b/app-packages/storm-v0_91/appConfig.json
@@ -3,7 +3,6 @@
   "metadata": {
   },
   "global": {
-    "agent.conf": "/slider/agent/conf/agent.ini",
     "application.def": "/slider/storm_v091.zip",
     "config_types": "storm-site",
     "java_home": "/usr/jdk64/jdk1.7.0_45",

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/adc7f4f1/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
index f3e2dd7..3835df6 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/agent/AgentClientProvider.java
@@ -18,6 +18,7 @@
 
 package org.apache.slider.providers.agent;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.slider.api.OptionKeys;
@@ -93,8 +94,10 @@ public class AgentClientProvider extends AbstractClientProvider
     sliderFileSystem.verifyFileExists(appDefPath);
 
     String agentConf = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().getMandatoryOption(AgentKeys.AGENT_CONF);
-    sliderFileSystem.verifyFileExists(new Path(agentConf));
+        getGlobalOptions().getOption(AgentKeys.AGENT_CONF, "");
+    if (StringUtils.isNotEmpty(agentConf)) {
+      sliderFileSystem.verifyFileExists(new Path(agentConf));
+    }
 
     String appHome = instanceDefinition.getAppConfOperations().
         getGlobalOptions().get(AgentKeys.PACKAGE_PATH);
@@ -173,15 +176,6 @@ public class AgentClientProvider extends AbstractClientProvider
                                    OptionKeys.INTERNAL_APPLICATION_IMAGE_PATH
                                    + " must be provided.");
     }
-
-    try {
-      // Validate the agent config
-      instanceDefinition.getAppConfOperations().
-          getGlobalOptions().getMandatoryOption(AgentKeys.AGENT_CONF);
-    } catch (BadConfigException bce) {
-      throw new BadConfigException("Agent config "+ AgentKeys.AGENT_CONF 
-                                   + " property must be provided.");
-    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/adc7f4f1/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 019a8e6..96acd41 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
@@ -28,10 +28,6 @@ public interface AgentKeys {
   /**
    * {@value}
    */
-  String CONF_FILE = "agent.conf";
-  /**
-   * {@value}
-   */
   String REGION_SERVER = "regionserver";
   /**
    * What is the command for hbase to print a version: {@value}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/adc7f4f1/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 6cd3d9e..3430baf 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
@@ -234,11 +234,13 @@ public class AgentProviderService extends AbstractProviderService implements
     launcher.addLocalResource(AgentKeys.APP_DEFINITION_DIR, appDefRes);
 
     String agentConf = instanceDefinition.getAppConfOperations().
-        getGlobalOptions().getMandatoryOption(AgentKeys.AGENT_CONF);
-    LocalResource agentConfRes = fileSystem.createAmResource(
-        fileSystem.getFileSystem().resolvePath(new Path(agentConf)),
-        LocalResourceType.FILE);
-    launcher.addLocalResource(AgentKeys.AGENT_CONFIG_FILE, agentConfRes);
+        getGlobalOptions().getOption(AgentKeys.AGENT_CONF, "");
+    if (org.apache.commons.lang.StringUtils.isNotEmpty(agentConf)) {
+      LocalResource agentConfRes = fileSystem.createAmResource(fileSystem
+          .getFileSystem().resolvePath(new Path(agentConf)),
+          LocalResourceType.FILE);
+      launcher.addLocalResource(AgentKeys.AGENT_CONFIG_FILE, agentConfRes);
+    }
 
     String agentVer = instanceDefinition.getAppConfOperations().
         getGlobalOptions().getOption(AgentKeys.AGENT_VERSION, null);


[47/50] [abbrv] git commit: SLIDER-107 updating release POMs for 0.40

Posted by sm...@apache.org.
SLIDER-107 updating release POMs for 0.40


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

Branch: refs/heads/master
Commit: e09a6a84e400d7648ee224d83490bb1d17f58652
Parents: bedf1c1
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Thu Jul 10 20:26:21 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Thu Jul 10 20:26:21 2014 -0700

----------------------------------------------------------------------
 app-packages/accumulo/pom.xml                              | 2 +-
 app-packages/command-logger/application-pkg/pom.xml        | 2 +-
 app-packages/command-logger/slider-pkg/pom.xml             | 2 +-
 pom.xml                                                    | 2 +-
 slider-agent/pom.xml                                       | 2 +-
 slider-assembly/pom.xml                                    | 2 +-
 slider-core/pom.xml                                        | 2 +-
 slider-funtest/pom.xml                                     | 2 +-
 slider-install/pom.xml                                     | 2 +-
 slider-providers/accumulo/accumulo-funtests/pom.xml        | 2 +-
 slider-providers/accumulo/slider-accumulo-provider/pom.xml | 2 +-
 slider-providers/hbase/hbase-funtests/pom.xml              | 2 +-
 slider-providers/hbase/slider-hbase-provider/pom.xml       | 2 +-
 13 files changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/app-packages/accumulo/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/pom.xml b/app-packages/accumulo/pom.xml
index d23933b..45dfd87 100644
--- a/app-packages/accumulo/pom.xml
+++ b/app-packages/accumulo/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/app-packages/command-logger/application-pkg/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/application-pkg/pom.xml b/app-packages/command-logger/application-pkg/pom.xml
index 7da7259..53f7fd2 100644
--- a/app-packages/command-logger/application-pkg/pom.xml
+++ b/app-packages/command-logger/application-pkg/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
     <relativePath>../../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/app-packages/command-logger/slider-pkg/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/pom.xml b/app-packages/command-logger/slider-pkg/pom.xml
index 42ec8da..0971868 100644
--- a/app-packages/command-logger/slider-pkg/pom.xml
+++ b/app-packages/command-logger/slider-pkg/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
     <relativePath>../../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a4489bb..a9a90ab 100644
--- a/pom.xml
+++ b/pom.xml
@@ -19,7 +19,7 @@
   <groupId>org.apache.slider</groupId>
   <artifactId>slider</artifactId>
   <name>Slider</name>
-  <version>0.31.0-incubating-SNAPSHOT</version>
+  <version>0.40</version>
   <packaging>pom</packaging>
 
   <description>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-agent/pom.xml
----------------------------------------------------------------------
diff --git a/slider-agent/pom.xml b/slider-agent/pom.xml
index efc61ef..7a3b447 100644
--- a/slider-agent/pom.xml
+++ b/slider-agent/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>slider-agent</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/slider-assembly/pom.xml b/slider-assembly/pom.xml
index 2a312bb..0ec91dd 100644
--- a/slider-assembly/pom.xml
+++ b/slider-assembly/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
   </parent>
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-core/pom.xml
----------------------------------------------------------------------
diff --git a/slider-core/pom.xml b/slider-core/pom.xml
index b6120fb..95a7e71 100644
--- a/slider-core/pom.xml
+++ b/slider-core/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
   </parent>
 
   <build>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-funtest/pom.xml
----------------------------------------------------------------------
diff --git a/slider-funtest/pom.xml b/slider-funtest/pom.xml
index 4b2edc5..cb16669 100644
--- a/slider-funtest/pom.xml
+++ b/slider-funtest/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
   </parent>
   <properties>
     <work.dir>package-tmp</work.dir>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-install/pom.xml
----------------------------------------------------------------------
diff --git a/slider-install/pom.xml b/slider-install/pom.xml
index df061a7..b08895f 100644
--- a/slider-install/pom.xml
+++ b/slider-install/pom.xml
@@ -28,7 +28,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-providers/accumulo/accumulo-funtests/pom.xml
----------------------------------------------------------------------
diff --git a/slider-providers/accumulo/accumulo-funtests/pom.xml b/slider-providers/accumulo/accumulo-funtests/pom.xml
index 27d9e89..1b25b33 100644
--- a/slider-providers/accumulo/accumulo-funtests/pom.xml
+++ b/slider-providers/accumulo/accumulo-funtests/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
     <relativePath>../../../</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-providers/accumulo/slider-accumulo-provider/pom.xml
----------------------------------------------------------------------
diff --git a/slider-providers/accumulo/slider-accumulo-provider/pom.xml b/slider-providers/accumulo/slider-accumulo-provider/pom.xml
index 00d2cce..cabea00 100644
--- a/slider-providers/accumulo/slider-accumulo-provider/pom.xml
+++ b/slider-providers/accumulo/slider-accumulo-provider/pom.xml
@@ -28,7 +28,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
     <relativePath>../../../</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-providers/hbase/hbase-funtests/pom.xml
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/hbase-funtests/pom.xml b/slider-providers/hbase/hbase-funtests/pom.xml
index cfebfc2..f5a155b 100644
--- a/slider-providers/hbase/hbase-funtests/pom.xml
+++ b/slider-providers/hbase/hbase-funtests/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
     <relativePath>../../../</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-providers/hbase/slider-hbase-provider/pom.xml
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/slider-hbase-provider/pom.xml b/slider-providers/hbase/slider-hbase-provider/pom.xml
index 1cd5e66..381a1b6 100644
--- a/slider-providers/hbase/slider-hbase-provider/pom.xml
+++ b/slider-providers/hbase/slider-hbase-provider/pom.xml
@@ -29,7 +29,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
     <relativePath>../../../</relativePath>
   </parent>
 


[23/50] [abbrv] git commit: SLIDER-217 Slider app-package for hbase is not generating the zip file

Posted by sm...@apache.org.
SLIDER-217 Slider app-package for hbase is not generating the zip file


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

Branch: refs/heads/master
Commit: 0007cb210affbdbd8fa70e80b3b673a4474ce9b1
Parents: 9be2bdd
Author: tedyu <yu...@gmail.com>
Authored: Tue Jul 8 07:27:06 2014 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Jul 8 07:27:06 2014 -0700

----------------------------------------------------------------------
 app-packages/hbase/src/assembly/hbase.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/0007cb21/app-packages/hbase/src/assembly/hbase.xml
----------------------------------------------------------------------
diff --git a/app-packages/hbase/src/assembly/hbase.xml b/app-packages/hbase/src/assembly/hbase.xml
index 143c88f..00b69ff 100644
--- a/app-packages/hbase/src/assembly/hbase.xml
+++ b/app-packages/hbase/src/assembly/hbase.xml
@@ -23,7 +23,7 @@
   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>dir</format>
+    <format>zip</format>
   </formats>
   <includeBaseDirectory>false</includeBaseDirectory>
 


[14/50] [abbrv] git commit: Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-slider into develop

Posted by sm...@apache.org.
Merge branch 'develop' of https://git-wip-us.apache.org/repos/asf/incubator-slider into develop


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

Branch: refs/heads/master
Commit: a613e20e760efc83aab1cd30fb495cd809dbeff9
Parents: 0b339b7 87fe586
Author: Steve Loughran <st...@apache.org>
Authored: Mon Jul 7 15:29:32 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Mon Jul 7 15:29:32 2014 +0100

----------------------------------------------------------------------
 app-packages/hbase/README.txt | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------



[26/50] [abbrv] git commit: Exclude output under app-packages/hbase/target from RAT check

Posted by sm...@apache.org.
Exclude output under app-packages/hbase/target from RAT check


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

Branch: refs/heads/master
Commit: 6f677f6a05f79af3d5484a0ec884ce24457926c5
Parents: adc7f4f
Author: tedyu <yu...@gmail.com>
Authored: Tue Jul 8 12:46:59 2014 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Jul 8 12:46:59 2014 -0700

----------------------------------------------------------------------
 pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/6f677f6a/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 6c86b33..49e8ce7 100644
--- a/pom.xml
+++ b/pom.xml
@@ -302,6 +302,7 @@
             <exclude>**/*.iml</exclude>
             <exclude>**/rat.txt</exclude>
             <exclude>DISCLAIMER</exclude>
+            <exclude>app-packages/hbase/target/**</exclude>
           </excludes>
         </configuration>
       </plugin>


[42/50] [abbrv] git commit: SLIDER-159 TestLongLivedProcess.testEcho failing on jenkins. Rework of processing to make sure that all lines of output get recorded on process exit

Posted by sm...@apache.org.
SLIDER-159 TestLongLivedProcess.testEcho failing on jenkins. Rework of processing to make sure that all lines of output get recorded on process exit


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

Branch: refs/heads/master
Commit: 66c677d32a5007d974e98bdecc0ccce89b3ef967
Parents: c011e7a
Author: Steve Loughran <st...@apache.org>
Authored: Thu Jul 10 11:15:41 2014 +0100
Committer: Steve Loughran <st...@apache.org>
Committed: Thu Jul 10 11:15:41 2014 +0100

----------------------------------------------------------------------
 .../services/workflow/LongLivedProcess.java     | 78 +++++++++++---------
 1 file changed, 43 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/66c677d3/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java b/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
index 05a1c50..ecc26b9 100644
--- a/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
+++ b/slider-core/src/main/java/org/apache/slider/server/services/workflow/LongLivedProcess.java
@@ -98,9 +98,9 @@ public class LongLivedProcess implements Runnable {
   private static final Logger LOG = LoggerFactory.getLogger(LongLivedProcess.class);
 
   /**
-   * Volatile flag to indicate that the process is done
+   *  flag to indicate that the process is done
    */
-  private volatile boolean finished;
+  private final AtomicBoolean finished = new AtomicBoolean(false);
 
   public LongLivedProcess(String name,
       Logger processLog,
@@ -202,7 +202,7 @@ public class LongLivedProcess implements Runnable {
    * @return true iff the process has been started and is not yet finished
    */
   public boolean isRunning() {
-    return process != null && !finished;
+    return process != null && !finished.get();
   }
 
   /**
@@ -301,7 +301,7 @@ public class LongLivedProcess implements Runnable {
       //here the process has finished
       LOG.debug("process {} has finished", name);
       //tell the logger it has to finish too
-      finished = true;
+      finished.set(true);
 
       // shut down the threads
       logExecutor.shutdown();
@@ -397,9 +397,10 @@ public class LongLivedProcess implements Runnable {
    * something that is only called once per line of IO?
    * @param line line to record
    * @param isErrorStream is the line from the error stream
+   * @param logger logger to log to
    */
   private synchronized void recordRecentLine(String line,
-                                             boolean isErrorStream) {
+      boolean isErrorStream, Logger logger) {
     if (line == null) {
       return;
     }
@@ -408,6 +409,11 @@ public class LongLivedProcess implements Runnable {
     if (recentLines.size() > recentLineLimit) {
       recentLines.remove(0);
     }
+    if (isErrorStream) {
+      logger.warn(line);
+    } else {
+      logger.info(line);
+    }
   }
 
   /**
@@ -485,23 +491,19 @@ public class LongLivedProcess implements Runnable {
             new InputStreamReader(process.getErrorStream()));
         outReader = new BufferedReader(
             new InputStreamReader(process.getInputStream()));
-        while (!finished) {
+        while (!finished.get()) {
           boolean processed = false;
           if (readAnyLine(errReader, errorLine, LINE_LENGTH)) {
-            String line = errorLine.toString();
-            recordRecentLine(line, true);
-            streamLog.warn(line);
+            recordRecentLine(errorLine.toString(), true, streamLog);
             errorLine.setLength(0);
             processed = true;
           }
           if (readAnyLine(outReader, outLine, LINE_LENGTH)) {
-            String line = outLine.toString();
-            recordRecentLine(line, false);
-            streamLog.info(line);
+            recordRecentLine(outLine.toString(), false, streamLog);
             outLine.setLength(0);
             processed |= true;
           }
-          if (!processed && !finished) {
+          if (!processed && !finished.get()) {
             //nothing processed: wait a bit for data.
             try {
               Thread.sleep(sleepTime);
@@ -514,37 +516,43 @@ public class LongLivedProcess implements Runnable {
         // finished: cleanup
 
         //print the current error line then stream through the rest
-        streamLog.error(errorLine.toString());
-        String line = errReader.readLine();
-        while (line != null) {
-          streamLog.error(line);
-          if (Thread.interrupted()) {
-            break;
-          }
-          line = errReader.readLine();
-          recordRecentLine(line, true);
-        }
+        recordFinalOutput(errReader, errorLine, true, streamLog);
         //now do the info line
-        streamLog.info(outLine.toString());
-        line = outReader.readLine();
-        while (line != null) {
-          streamLog.info(line);
-          if (Thread.interrupted()) {
-            break;
-          }
-          line = outReader.readLine();
-          recordRecentLine(line, false);
-        }
+        recordFinalOutput(outReader, outLine, false, streamLog);
 
       } catch (Exception ignored) {
         LOG.warn("encountered {}", ignored, ignored);
         //process connection has been torn down
       } finally {
-        //mark output as done
-        finalOutputProcessed.set(true);
         // close streams
         IOUtils.closeStream(errReader);
         IOUtils.closeStream(outReader);
+        //mark output as done
+        finalOutputProcessed.set(true);
+      }
+    }
+
+    /**
+     * Record the final output of a process stream
+     * @param reader reader of output
+     * @param lineBuilder string builder into which line is built
+     * @param isErrorStream flag to indicate whether or not this is the
+     * is the line from the error stream
+     * @param logger logger to log to
+     * @throws IOException
+     */
+    protected void recordFinalOutput(BufferedReader reader,
+        StringBuilder lineBuilder, boolean isErrorStream, Logger logger) throws
+        IOException {
+      String line = lineBuilder.toString();
+      recordRecentLine(line, isErrorStream, logger);
+      line = reader.readLine();
+      while (line != null) {
+        recordRecentLine(line, isErrorStream, logger);
+        line = reader.readLine();
+        if (Thread.interrupted()) {
+          break;
+        }
       }
     }
   }


[09/50] [abbrv] git commit: SLIDER-178 created basic functional tests for accumulo app package

Posted by sm...@apache.org.
SLIDER-178 created basic functional tests for accumulo app package


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

Branch: refs/heads/master
Commit: f94e10393eb76888ad2b40dae838cfa8cfb308f2
Parents: 7823d3a
Author: Billie Rinaldi <bi...@gmail.com>
Authored: Wed Jul 2 14:16:39 2014 -0700
Committer: Billie Rinaldi <bi...@gmail.com>
Committed: Wed Jul 2 14:16:39 2014 -0700

----------------------------------------------------------------------
 app-packages/accumulo/appConfig.json            |   4 +-
 app-packages/accumulo/pom.xml                   | 107 ++++++++-
 app-packages/accumulo/resources.json            |  15 +-
 app-packages/accumulo/src/assembly/accumulo.xml |   1 +
 .../AccumuloAgentCommandTestBase.groovy         |  52 ++++
 .../funtest/accumulo/AccumuloBasicIT.groovy     | 117 +++++++++
 .../accumulo/AccumuloMonitorSSLIT.groovy        |  73 ++++++
 .../funtest/accumulo/AccumuloReadWriteIT.groovy | 116 +++++++++
 .../StubToForceGroovyTestsToCompile.java        |  22 ++
 .../test/resources/appConfig_monitor_ssl.json   |  62 +++++
 .../accumulo/src/test/resources/resources.json  |  36 +++
 .../slider/test/YarnMiniClusterTestBase.groovy  |   2 +-
 slider-funtest/pom.xml                          |   5 +-
 .../framework/AgentCommandTestBase.groovy       | 237 +++++++++++++++++++
 .../accumulo/StubToForceGroovySrcToCompile.java |  22 ++
 .../lifecycle/AgentCommandTestBase.groovy       | 233 ------------------
 .../lifecycle/TestAgentClusterLifecycle.groovy  |   1 +
 .../funtest/lifecycle/TestAgentFailures.groovy  |   1 +
 .../funtest/lifecycle/TestAgentFailures2.groovy |   1 +
 .../lifecycle/TestAppsThroughAgent.groovy       |   1 +
 .../lifecycle/TestClusterBuildDestroy.groovy    |   1 +
 .../accumulo/AccumuloProviderService.java       |   4 +
 22 files changed, 870 insertions(+), 243 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/app-packages/accumulo/appConfig.json
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/appConfig.json b/app-packages/accumulo/appConfig.json
index b8d80d1..6a22336 100644
--- a/app-packages/accumulo/appConfig.json
+++ b/app-packages/accumulo/appConfig.json
@@ -3,8 +3,8 @@
   "metadata": {
   },
   "global": {
-    "agent.conf": "/slider/agent/conf/agent.ini",
-    "application.def": "/slider/${app.package.name}.zip",
+    "agent.conf": "agent.ini",
+    "application.def": "${app.package.name}.zip",
     "config_types": "accumulo-site",
     "java_home": "/usr/jdk64/jdk1.7.0_45",
     "package_list": "files/accumulo-${accumulo.version}-bin.tar.gz",

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/app-packages/accumulo/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/pom.xml b/app-packages/accumulo/pom.xml
index 9a474e3..d23933b 100644
--- a/app-packages/accumulo/pom.xml
+++ b/app-packages/accumulo/pom.xml
@@ -24,9 +24,10 @@
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>slider-accumulo-app-package</artifactId>
-  <packaging>pom</packaging>
+  <packaging>jar</packaging>
   <name>Slider Accumulo App Package</name>
   <description>Slider Accumulo App Package</description>
+
   <properties>
     <work.dir>package-tmp</work.dir>
     <app.package.name>apache-slider-accumulo-${accumulo.version}-app-package-${project.version}</app.package.name>
@@ -70,16 +71,90 @@
                 <configuration>
                   <includeArtifactIds>accumulo</includeArtifactIds>
                   <includeTypes>tar.gz</includeTypes>
+                  <excludeTransitive>true</excludeTransitive>
                   <outputDirectory>${project.build.directory}/${work.dir}</outputDirectory>
                 </configuration>
               </execution>
             </executions>
           </plugin>
+
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-failsafe-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>run-integration-tests</id>
+                <goals>
+                  <goal>integration-test</goal>
+                  <goal>verify</goal>
+                </goals>
+              </execution>
+            </executions>
+            <configuration>
+              <systemPropertyVariables>
+                <java.net.preferIPv4Stack>true</java.net.preferIPv4Stack>
+                <java.awt.headless>true</java.awt.headless>
+                <!-- this property must be supplied-->
+                <slider.conf.dir>${slider.conf.dir}</slider.conf.dir>
+                <slider.bin.dir>../../slider-assembly/target/slider-${project.version}-all/slider-${project.version}</slider.bin.dir>
+                <test.app.pkg.dir>target</test.app.pkg.dir>
+                <test.app.pkg.file>${app.package.name}.zip</test.app.pkg.file>
+                <test.app.resource>target/test-config/resources.json</test.app.resource>
+                <test.app.template>target/${app.package.name}/appConfig.json</test.app.template>
+              </systemPropertyVariables>
+            </configuration>
+          </plugin>
         </plugins>
       </build>
     </profile>
   </profiles>
 
+  <build>
+    <!-- resources are filtered for dynamic updates. This gets build info in-->
+    <resources>
+      <resource>
+        <directory>src/test/resources</directory>
+        <filtering>true</filtering>
+        <targetPath>${project.build.directory}/test-config</targetPath>
+      </resource>
+    </resources>
+
+    <plugins>
+      <plugin>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <version>${maven-compiler-plugin.version}</version>
+        <configuration>
+          <compilerId>groovy-eclipse-compiler</compilerId>
+          <!-- set verbose to be true if you want lots of uninteresting messages -->
+          <!-- <verbose>true</verbose> -->
+          <source>${project.java.src.version}</source>
+          <target>${project.java.src.version}</target>
+        </configuration>
+        <dependencies>
+          <dependency>
+            <groupId>org.codehaus.groovy</groupId>
+            <artifactId>groovy-eclipse-compiler</artifactId>
+            <version>${groovy-eclipse-compiler.version}</version>
+          </dependency>
+          <dependency>
+            <groupId>org.codehaus.groovy</groupId>
+            <artifactId>groovy-eclipse-batch</artifactId>
+            <version>${groovy-eclipse-batch.version}</version>
+          </dependency>
+        </dependencies>
+      </plugin>
+
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <!-- can't figure out how to get the surefire plugin not to pick up the ITs, so skip it entirely -->
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+
   <dependencies>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
@@ -88,6 +163,36 @@
       <classifier>bin</classifier>
       <type>tar.gz</type>
     </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.accumulo</groupId>
+      <artifactId>accumulo-test</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.slider</groupId>
+      <artifactId>slider-core</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.slider</groupId>
+      <artifactId>slider-funtest</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.codehaus.groovy</groupId>
+      <artifactId>groovy-all</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/app-packages/accumulo/resources.json
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/resources.json b/app-packages/accumulo/resources.json
index 54d054b..f876901 100644
--- a/app-packages/accumulo/resources.json
+++ b/app-packages/accumulo/resources.json
@@ -7,25 +7,30 @@
   "components": {
     "ACCUMULO_MASTER": {
       "yarn.role.priority": "1",
-      "yarn.component.instances": "1"
+      "yarn.component.instances": "1",
+      "yarn.memory": "256"
     },
     "slider-appmaster": {
     },
     "ACCUMULO_TSERVER": {
       "yarn.role.priority": "2",
-      "yarn.component.instances": "1"
+      "yarn.component.instances": "1",
+      "yarn.memory": "256"
     },
     "ACCUMULO_MONITOR": {
       "yarn.role.priority": "3",
-      "yarn.component.instances": "1"
+      "yarn.component.instances": "1",
+      "yarn.memory": "128"
     },
     "ACCUMULO_GC": {
       "yarn.role.priority": "4",
-      "yarn.component.instances": "1"
+      "yarn.component.instances": "1",
+      "yarn.memory": "128"
     },
     "ACCUMULO_TRACER": {
       "yarn.role.priority": "5",
-      "yarn.component.instances": "1"
+      "yarn.component.instances": "1",
+      "yarn.memory": "256"
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/app-packages/accumulo/src/assembly/accumulo.xml
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/assembly/accumulo.xml b/app-packages/accumulo/src/assembly/accumulo.xml
index 67e0cbb..a8f9578 100644
--- a/app-packages/accumulo/src/assembly/accumulo.xml
+++ b/app-packages/accumulo/src/assembly/accumulo.xml
@@ -24,6 +24,7 @@
   <id>accumulo_v${accumulo.version}</id>
   <formats>
     <format>zip</format>
+    <format>dir</format>
   </formats>
   <includeBaseDirectory>false</includeBaseDirectory>
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloAgentCommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloAgentCommandTestBase.groovy b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloAgentCommandTestBase.groovy
new file mode 100644
index 0000000..50ecfcd
--- /dev/null
+++ b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloAgentCommandTestBase.groovy
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.slider.funtest.accumulo
+
+import groovy.util.logging.Slf4j
+import org.apache.slider.funtest.framework.AgentCommandTestBase
+import org.junit.After
+import org.junit.Before
+
+@Slf4j
+abstract class AccumuloAgentCommandTestBase extends AgentCommandTestBase {
+  protected static final int ACCUMULO_LAUNCH_WAIT_TIME
+  protected static final int ACCUMULO_GO_LIVE_TIME = 60000
+
+  // parameters must match those found in the default appConfig.json
+  protected static final String INSTANCE_NAME = "instancename"
+  protected static final String USER = "root"
+  protected static final String PASSWORD = "secret"
+
+  static {
+    ACCUMULO_LAUNCH_WAIT_TIME = getTimeOptionMillis(SLIDER_CONFIG,
+      KEY_ACCUMULO_LAUNCH_TIME,
+      1000 * DEFAULT_ACCUMULO_LAUNCH_TIME_SECONDS)
+  }
+
+  abstract public String getClusterName();
+
+  @Before
+  public void prepareCluster() {
+    setupCluster(getClusterName())
+  }
+
+  @After
+  public void destroyCluster() {
+    cleanup(getClusterName())
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloBasicIT.groovy
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloBasicIT.groovy b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloBasicIT.groovy
new file mode 100644
index 0000000..bcb952b
--- /dev/null
+++ b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloBasicIT.groovy
@@ -0,0 +1,117 @@
+/*
+ * 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.funtest.accumulo
+
+import groovy.util.logging.Slf4j
+import org.apache.slider.api.ClusterDescription
+import org.apache.slider.client.SliderClient
+import org.apache.slider.common.SliderKeys
+import org.apache.slider.core.registry.docstore.PublishedConfiguration
+import org.apache.slider.core.registry.info.ServiceInstanceData
+import org.apache.slider.core.registry.retrieve.RegistryRetriever
+import org.apache.slider.funtest.framework.SliderShell
+import org.apache.slider.server.services.curator.CuratorServiceInstance
+import org.junit.Test
+
+@Slf4j
+class AccumuloBasicIT extends AccumuloAgentCommandTestBase {
+
+  @Override
+  public String getClusterName() {
+    return "test_accumulo_basic"
+  }
+
+  @Test
+  public void testAccumuloClusterCreate() throws Throwable {
+
+    describe getDescription()
+
+    def path = buildClusterPath(getClusterName())
+    assert !clusterFS.exists(path)
+
+    SliderShell shell = slider(EXIT_SUCCESS,
+      [
+        ACTION_CREATE, getClusterName(),
+        ARG_IMAGE, agentTarballPath.toString(),
+        ARG_TEMPLATE, APP_TEMPLATE,
+        ARG_RESOURCES, APP_RESOURCE
+      ])
+
+    logShell(shell)
+
+    ensureApplicationIsUp(getClusterName())
+
+    // must match the values in src/test/resources/resources.json
+    Map<String, Integer> roleMap = [
+      "ACCUMULO_MASTER" : 1,
+      "ACCUMULO_TSERVER" : 2,
+      "ACCUMULO_MONITOR": 1,
+      "ACCUMULO_GC": 0,
+      "ACCUMULO_TRACER" : 0
+    ];
+
+    //get a slider client against the cluster
+    SliderClient sliderClient = bondToCluster(SLIDER_CONFIG, getClusterName())
+    ClusterDescription cd = sliderClient.clusterDescription
+    assert getClusterName() == cd.name
+
+    log.info("Connected via Client {}", sliderClient.toString())
+
+    //wait for the role counts to be reached
+    waitForRoleCount(sliderClient, roleMap, ACCUMULO_LAUNCH_WAIT_TIME)
+
+    sleep(ACCUMULO_GO_LIVE_TIME)
+
+    clusterLoadOperations(cd, sliderClient)
+  }
+
+
+  public String getDescription() {
+    return "Create a working Accumulo cluster $clusterName"
+  }
+
+  public static String getMonitorUrl(SliderClient sliderClient, String clusterName) {
+    CuratorServiceInstance<ServiceInstanceData> instance =
+      sliderClient.getRegistry().queryForInstance(SliderKeys.APP_TYPE, clusterName)
+    ServiceInstanceData serviceInstanceData = instance.payload
+    RegistryRetriever retriever = new RegistryRetriever(serviceInstanceData)
+    PublishedConfiguration configuration = retriever.retrieveConfiguration(
+      retriever.getConfigurations(true), "quicklinks", true)
+
+    // must match name set in metainfo.xml
+    String monitorUrl = configuration.entries.get("org.apache.slider.monitor")
+
+    assertNotNull monitorUrl
+    return monitorUrl
+  }
+
+  public static void checkMonitorPage(String monitorUrl) {
+    String monitor = fetchWebPageWithoutError(monitorUrl);
+    assume monitor != null, "Monitor page null"
+    assume monitor.length() > 100, "Monitor page too short"
+    assume monitor.contains("Accumulo Overview"), "Monitor page didn't contain expected text"
+  }
+
+  /**
+   * Override point for any cluster load operations
+   */
+  public void clusterLoadOperations(ClusterDescription cd, SliderClient sliderClient) {
+    String monitorUrl = getMonitorUrl(sliderClient, getClusterName())
+    assert monitorUrl.startsWith("http://"), "Monitor URL didn't have expected protocol"
+    checkMonitorPage(monitorUrl)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloMonitorSSLIT.groovy
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloMonitorSSLIT.groovy b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloMonitorSSLIT.groovy
new file mode 100644
index 0000000..6f68e13
--- /dev/null
+++ b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloMonitorSSLIT.groovy
@@ -0,0 +1,73 @@
+/*
+ * 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.funtest.accumulo
+
+import groovy.util.logging.Slf4j
+import org.apache.slider.api.ClusterDescription
+import org.apache.slider.client.SliderClient
+
+import javax.net.ssl.KeyManager
+import javax.net.ssl.SSLContext
+import javax.net.ssl.TrustManager
+import javax.net.ssl.X509TrustManager
+import java.security.SecureRandom
+import java.security.cert.CertificateException
+import java.security.cert.X509Certificate
+
+@Slf4j
+class AccumuloMonitorSSLIT extends AccumuloBasicIT {
+  AccumuloMonitorSSLIT() {
+    APP_TEMPLATE = "target/test-config/appConfig_monitor_ssl.json"
+  }
+
+  @Override
+  public String getClusterName() {
+    return "test_monitor_ssl";
+  }
+
+  @Override
+  public String getDescription() {
+    return "Test enable monitor SSL $clusterName"
+  }
+
+  @Override
+  public void clusterLoadOperations(ClusterDescription cd, SliderClient sliderClient) {
+    String monitorUrl = getMonitorUrl(sliderClient, getClusterName())
+    assert monitorUrl.startsWith("https://"), "Monitor URL didn't have expected protocol"
+
+    SSLContext ctx = SSLContext.getInstance("SSL");
+    TrustManager[] t = new TrustManager[1];
+    t[0] = new DefaultTrustManager();
+    ctx.init(new KeyManager[0], t, new SecureRandom());
+    SSLContext.setDefault(ctx);
+    checkMonitorPage(monitorUrl)
+  }
+
+  private static class DefaultTrustManager implements X509TrustManager {
+    @Override
+    public void checkClientTrusted(X509Certificate[] arg0, String arg1) throws CertificateException {}
+
+    @Override
+    public void checkServerTrusted(X509Certificate[] arg0, String arg1) throws CertificateException {}
+
+    @Override
+    public X509Certificate[] getAcceptedIssuers() {
+      return null;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloReadWriteIT.groovy
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloReadWriteIT.groovy b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloReadWriteIT.groovy
new file mode 100644
index 0000000..cdbbcce
--- /dev/null
+++ b/app-packages/accumulo/src/test/groovy/org/apache/slider/funtest/accumulo/AccumuloReadWriteIT.groovy
@@ -0,0 +1,116 @@
+/*
+ * 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.funtest.accumulo
+
+import groovy.util.logging.Slf4j
+import org.apache.accumulo.core.cli.BatchWriterOpts
+import org.apache.accumulo.core.cli.ScannerOpts
+import org.apache.accumulo.core.client.Connector
+import org.apache.accumulo.core.client.ZooKeeperInstance
+import org.apache.accumulo.core.client.security.tokens.PasswordToken
+import org.apache.accumulo.test.TestIngest
+import org.apache.accumulo.test.VerifyIngest
+import org.apache.slider.api.ClusterDescription
+import org.apache.slider.client.SliderClient
+import org.apache.slider.common.SliderXmlConfKeys
+import org.apache.slider.funtest.framework.FuntestProperties
+
+import java.util.concurrent.atomic.AtomicBoolean
+
+@Slf4j
+class AccumuloReadWriteIT extends AccumuloBasicIT {
+
+  @Override
+  public String getClusterName() {
+    return "test_read_write";
+  }
+
+  @Override
+  public String getDescription() {
+    return "Test reading and writing to Accumulo cluster $clusterName"
+  }
+
+  @Override
+  public void clusterLoadOperations(ClusterDescription cd, SliderClient sliderClient) {
+    try {
+      String zookeepers = SLIDER_CONFIG.get(SliderXmlConfKeys.REGISTRY_ZK_QUORUM,
+        FuntestProperties.DEFAULT_SLIDER_ZK_HOSTS)
+
+      ZooKeeperInstance instance = new ZooKeeperInstance(INSTANCE_NAME, zookeepers)
+      Connector connector = instance.getConnector(USER, new PasswordToken(PASSWORD))
+
+      ingest(connector, 200000, 1, 50, 0);
+      verify(connector, 200000, 1, 50, 0);
+
+      ingest(connector, 2, 1, 500000, 0);
+      verify(connector, 2, 1, 500000, 0);
+
+      interleaveTest(connector);
+    } catch (Exception e) {
+      fail("Got exception connecting/reading/writing "+e)
+    }
+  }
+
+  public static void ingest(Connector connector, int rows, int cols, int width, int offset) throws Exception {
+    TestIngest.Opts opts = new TestIngest.Opts();
+    opts.rows = rows;
+    opts.cols = cols;
+    opts.dataSize = width;
+    opts.startRow = offset;
+    opts.columnFamily = "colf";
+    opts.createTable = true;
+    TestIngest.ingest(connector, opts, new BatchWriterOpts());
+  }
+
+  private static void verify(Connector connector, int rows, int cols, int width, int offset) throws Exception {
+    ScannerOpts scannerOpts = new ScannerOpts();
+    VerifyIngest.Opts opts = new VerifyIngest.Opts();
+    opts.rows = rows;
+    opts.cols = cols;
+    opts.dataSize = width;
+    opts.startRow = offset;
+    opts.columnFamily = "colf";
+    VerifyIngest.verifyIngest(connector, opts, scannerOpts);
+  }
+
+  static void interleaveTest(final Connector connector) throws Exception {
+    final int ROWS = 200000;
+    final AtomicBoolean fail = new AtomicBoolean(false);
+    final int CHUNKSIZE = ROWS / 10;
+    ingest(connector, CHUNKSIZE, 1, 50, 0);
+    int i;
+    for (i = 0; i < ROWS; i += CHUNKSIZE) {
+      final int start = i;
+      Thread verify = new Thread() {
+        @Override
+        public void run() {
+          try {
+            verify(connector, CHUNKSIZE, 1, 50, start);
+          } catch (Exception ex) {
+            fail.set(true);
+          }
+        }
+      };
+      ingest(connector, CHUNKSIZE, 1, 50, i + CHUNKSIZE);
+      verify.join();
+      assertFalse(fail.get());
+    }
+    verify(connector, CHUNKSIZE, 1, 50, i);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/app-packages/accumulo/src/test/java/org/apache/slider/funtest/accumulo/StubToForceGroovyTestsToCompile.java
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/test/java/org/apache/slider/funtest/accumulo/StubToForceGroovyTestsToCompile.java b/app-packages/accumulo/src/test/java/org/apache/slider/funtest/accumulo/StubToForceGroovyTestsToCompile.java
new file mode 100644
index 0000000..7d289e0
--- /dev/null
+++ b/app-packages/accumulo/src/test/java/org/apache/slider/funtest/accumulo/StubToForceGroovyTestsToCompile.java
@@ -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.
+ */
+
+package org.apache.slider.funtest.accumulo;
+
+public class StubToForceGroovyTestsToCompile {
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/app-packages/accumulo/src/test/resources/appConfig_monitor_ssl.json
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/test/resources/appConfig_monitor_ssl.json b/app-packages/accumulo/src/test/resources/appConfig_monitor_ssl.json
new file mode 100644
index 0000000..8b63d06
--- /dev/null
+++ b/app-packages/accumulo/src/test/resources/appConfig_monitor_ssl.json
@@ -0,0 +1,62 @@
+{
+  "schema": "http://example.org/specification/v2.0.0",
+  "metadata": {
+  },
+  "global": {
+    "agent.conf": "agent.ini",
+    "application.def": "${app.package.name}.zip",
+    "config_types": "accumulo-site",
+    "java_home": "/usr/jdk64/jdk1.7.0_45",
+    "package_list": "files/accumulo-${accumulo.version}-bin.tar.gz",
+    "site.global.app_user": "yarn",
+    "site.global.app_log_dir": "${AGENT_LOG_ROOT}/app/log",
+    "site.global.app_pid_dir": "${AGENT_WORK_ROOT}/app/run",
+    "site.global.app_root": "${AGENT_WORK_ROOT}/app/install/accumulo-${accumulo.version}",
+    "site.global.app_install_dir": "${AGENT_WORK_ROOT}/app/install",
+    "site.global.tserver_heapsize": "128m",
+    "site.global.master_heapsize": "128m",
+    "site.global.monitor_heapsize": "64m",
+    "site.global.gc_heapsize": "64m",
+    "site.global.other_heapsize": "128m",
+    "site.global.hadoop_prefix": "/usr/lib/hadoop",
+    "site.global.hadoop_conf_dir": "/etc/hadoop/conf",
+    "site.global.zookeeper_home": "/usr/lib/zookeeper",
+    "site.global.accumulo_instance_name": "instancename",
+    "site.global.accumulo_root_password": "secret",
+    "site.global.user_group": "hadoop",
+    "site.global.security_enabled": "false",
+    "site.global.monitor_protocol": "https",
+    "site.accumulo-site.instance.volumes": "${DEFAULT_DATA_DIR}/data",
+    "site.accumulo-site.instance.zookeeper.host": "${ZK_HOST}",
+    "site.accumulo-site.instance.secret": "DEFAULT",
+    "site.accumulo-site.tserver.memory.maps.max": "80M",
+    "site.accumulo-site.tserver.cache.data.size": "7M",
+    "site.accumulo-site.tserver.cache.index.size": "20M",
+    "site.accumulo-site.trace.token.property.password": "secret",
+    "site.accumulo-site.trace.user": "root",
+    "site.accumulo-site.tserver.sort.buffer.size": "50M",
+    "site.accumulo-site.tserver.walog.max.size": "100M",
+    "site.accumulo-site.master.port.client": "0",
+    "site.accumulo-site.trace.port.client": "0",
+    "site.accumulo-site.tserver.port.client": "0",
+    "site.accumulo-site.gc.port.client": "0",
+    "site.accumulo-site.monitor.port.client": "${ACCUMULO_MONITOR.ALLOCATED_PORT}",
+    "site.accumulo-site.monitor.port.log4j": "0",
+    "site.accumulo-site.general.classpaths": "$ACCUMULO_HOME/lib/accumulo-server.jar,\n$ACCUMULO_HOME/lib/accumulo-core.jar,\n$ACCUMULO_HOME/lib/accumulo-start.jar,\n$ACCUMULO_HOME/lib/accumulo-fate.jar,\n$ACCUMULO_HOME/lib/accumulo-proxy.jar,\n$ACCUMULO_HOME/lib/[^.].*.jar,\n$ZOOKEEPER_HOME/zookeeper[^.].*.jar,\n$HADOOP_CONF_DIR,\n$HADOOP_PREFIX/[^.].*.jar,\n$HADOOP_PREFIX/lib/[^.].*.jar,\n$HADOOP_PREFIX/share/hadoop/common/.*.jar,\n$HADOOP_PREFIX/share/hadoop/common/lib/.*.jar,\n$HADOOP_PREFIX/share/hadoop/hdfs/.*.jar,\n$HADOOP_PREFIX/share/hadoop/mapreduce/.*.jar,\n$HADOOP_PREFIX/share/hadoop/yarn/.*.jar,\n/usr/lib/hadoop/.*.jar,\n/usr/lib/hadoop/lib/.*.jar,\n/usr/lib/hadoop-hdfs/.*.jar,\n/usr/lib/hadoop-mapreduce/.*.jar,\n/usr/lib/hadoop-yarn/.*.jar,"
+  },
+  "components": {
+    "ACCUMULO_MASTER": {
+    },
+    "slider-appmaster": {
+      "jvm.heapsize": "256M"
+    },
+    "ACCUMULO_TSERVER": {
+    },
+    "ACCUMULO_MONITOR": {
+    },
+    "ACCUMULO_GC": {
+    },
+    "ACCUMULO_TRACER": {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/app-packages/accumulo/src/test/resources/resources.json
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/src/test/resources/resources.json b/app-packages/accumulo/src/test/resources/resources.json
new file mode 100644
index 0000000..0d536aa
--- /dev/null
+++ b/app-packages/accumulo/src/test/resources/resources.json
@@ -0,0 +1,36 @@
+{
+  "schema": "http://example.org/specification/v2.0.0",
+  "metadata": {
+  },
+  "global": {
+  },
+  "components": {
+    "ACCUMULO_MASTER": {
+      "yarn.role.priority": "1",
+      "yarn.component.instances": "1",
+      "yarn.memory": "256"
+    },
+    "slider-appmaster": {
+    },
+    "ACCUMULO_TSERVER": {
+      "yarn.role.priority": "2",
+      "yarn.component.instances": "2",
+      "yarn.memory": "256"
+    },
+    "ACCUMULO_MONITOR": {
+      "yarn.role.priority": "3",
+      "yarn.component.instances": "1",
+      "yarn.memory": "128"
+    },
+    "ACCUMULO_GC": {
+      "yarn.role.priority": "4",
+      "yarn.component.instances": "0",
+      "yarn.memory": "128"
+    },
+    "ACCUMULO_TRACER": {
+      "yarn.role.priority": "5",
+      "yarn.component.instances": "0",
+      "yarn.memory": "256"
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy b/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
index 80a83a4..2c1b270 100644
--- a/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/test/YarnMiniClusterTestBase.groovy
@@ -183,7 +183,7 @@ public abstract class YarnMiniClusterTestBase extends ServiceLauncherBaseTest {
         hbaseLaunchWaitTime)
 
     accumuloTestsEnabled =
-        testConf.getBoolean(KEY_TEST_ACCUMULO_ENABLED, hbaseTestsEnabled)
+        testConf.getBoolean(KEY_TEST_ACCUMULO_ENABLED, accumuloTestsEnabled)
     accumuloLaunchWaitTime = getTimeOptionMillis(testConf,
         KEY_ACCUMULO_LAUNCH_TIME,
         accumuloLaunchWaitTime)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/slider-funtest/pom.xml
----------------------------------------------------------------------
diff --git a/slider-funtest/pom.xml b/slider-funtest/pom.xml
index 76785a0..4b2edc5 100644
--- a/slider-funtest/pom.xml
+++ b/slider-funtest/pom.xml
@@ -116,7 +116,10 @@
             <!-- this property must be supplied-->
             <slider.conf.dir>${slider.conf.dir}</slider.conf.dir>
             <slider.bin.dir>../slider-assembly/target/slider-${project.version}-all/slider-${project.version}</slider.bin.dir>
-            <test.app.pkg>target/package-tmp/apache-slider-command-logger.zip</test.app.pkg>
+            <test.app.pkg.dir>../app-packages/command-logger/slider-pkg/target</test.app.pkg.dir>
+            <test.app.pkg.file>apache-slider-command-logger.zip</test.app.pkg.file>
+            <test.app.resource>../slider-core/src/test/app_packages/test_command_log/resources.json</test.app.resource>
+            <test.app.template>../slider-core/src/test/app_packages/test_command_log/appConfig.json</test.app.template>
           </systemPropertyVariables>
           <includes>
             <include>**/Test*.java</include>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentCommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentCommandTestBase.groovy b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentCommandTestBase.groovy
new file mode 100644
index 0000000..26ae2bb
--- /dev/null
+++ b/slider-funtest/src/main/groovy/org/apache/slider/funtest/framework/AgentCommandTestBase.groovy
@@ -0,0 +1,237 @@
+/*
+ * 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.funtest.framework
+
+import groovy.util.logging.Slf4j
+import org.apache.hadoop.fs.Path
+import org.apache.slider.common.SliderExitCodes
+import org.apache.slider.common.params.Arguments
+import org.apache.slider.common.params.SliderActions
+import org.apache.slider.funtest.framework.AgentUploads
+import org.apache.slider.funtest.framework.CommandTestBase
+import org.apache.slider.funtest.framework.FuntestProperties
+import org.apache.slider.funtest.framework.SliderShell
+import org.apache.tools.zip.ZipEntry
+import org.apache.tools.zip.ZipOutputStream
+import org.junit.Before
+import org.junit.BeforeClass
+import org.junit.Rule
+import org.junit.rules.TemporaryFolder
+
+@Slf4j
+class AgentCommandTestBase extends CommandTestBase
+implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
+
+  public static final boolean AGENTTESTS_ENABLED
+  private static String TEST_APP_PKG_DIR_PROP = "test.app.pkg.dir"
+  private static String TEST_APP_PKG_FILE_PROP = "test.app.pkg.file"
+  private static String TEST_APP_RESOURCE = "test.app.resource"
+  private static String TEST_APP_TEMPLATE = "test.app.template"
+
+
+  protected String APP_RESOURCE = sysprop(TEST_APP_RESOURCE)
+  protected String APP_TEMPLATE = sysprop(TEST_APP_TEMPLATE)
+  public static final String TEST_APP_PKG_DIR = sysprop(TEST_APP_PKG_DIR_PROP)
+  public static final String TEST_APP_PKG_FILE = sysprop(TEST_APP_PKG_FILE_PROP)
+
+
+  protected static Path agentTarballPath;
+  protected static Path appPkgPath;
+  protected static Path agtIniPath;
+
+  protected static boolean setup_failed
+
+  static {
+    AGENTTESTS_ENABLED = SLIDER_CONFIG.getBoolean(KEY_TEST_AGENT_ENABLED, false)
+  }
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  public static void assumeAgentTestsEnabled() {
+    assumeFunctionalTestsEnabled()
+    assume(AGENTTESTS_ENABLED, "Agent tests disabled")
+  }
+
+  @BeforeClass
+  public static void setupAgent() {
+    assumeAgentTestsEnabled()
+
+  }
+
+  @Before
+  public void uploadAgentTarball() {
+    def agentUploads = new AgentUploads(SLIDER_CONFIG)
+    (agentTarballPath, agtIniPath) =
+        agentUploads.uploadAgentFiles(SLIDER_TAR_DIRECTORY, false)
+  }
+
+
+  @Before
+  public void setupApplicationPackage() {
+    try {
+      AgentUploads agentUploads = new AgentUploads(SLIDER_CONFIG)
+      agentUploads.uploader.mkHomeDir()
+
+      appPkgPath = new Path(clusterFS.homeDirectory, TEST_APP_PKG_FILE)
+      if (clusterFS.exists(appPkgPath)) {
+        clusterFS.delete(appPkgPath, false)
+        log.info "Existing app pkg deleted from $appPkgPath"
+      }
+
+      File zipFileName = new File(TEST_APP_PKG_DIR, TEST_APP_PKG_FILE).canonicalFile
+      agentUploads.uploader.copyIfOutOfDate(zipFileName, appPkgPath, false)
+      assume(clusterFS.exists(appPkgPath), "App pkg not uploaded to $appPkgPath")
+      log.info "App pkg uploaded at $appPkgPath"
+    } catch (Exception e) {
+      setup_failed = true
+      fail("Setup failed "+e)
+    }
+  }
+
+  public static void logShell(SliderShell shell) {
+    for (String str in shell.out) {
+      log.info str
+    }
+  }
+
+  public static void assertComponentCount(String component, int count, SliderShell shell) {
+    log.info("Asserting component count.")
+    String entry = findLineEntry(shell, ["instances", component] as String[])
+    log.info(entry)
+    assert entry != null
+    int instanceCount = 0
+    int index = entry.indexOf("container_")
+    while (index != -1) {
+      instanceCount++;
+      index = entry.indexOf("container_", index + 1)
+    }
+
+    assert instanceCount == count, 'Instance count for component did not match expected. Parsed: ' + entry
+  }
+
+  public static String findLineEntry(SliderShell shell, String[] locaters) {
+    int index = 0;
+    for (String str in shell.out) {
+      if (str.contains("\"" + locaters[index] + "\"")) {
+        if (locaters.size() == index + 1) {
+          return str;
+        } else {
+          index++;
+        }
+      }
+    }
+
+    return null;
+  }
+
+  public static String findLineEntryValue(SliderShell shell, String[] locaters) {
+    String line = findLineEntry(shell, locaters);
+
+    if (line != null) {
+      log.info("Parsing {} for value.", line)
+      int dividerIndex = line.indexOf(":");
+      if (dividerIndex > 0) {
+        String value = line.substring(dividerIndex + 1).trim()
+        if (value.endsWith(",")) {
+          value = value.subSequence(0, value.length() - 1)
+        }
+        return value;
+      }
+    }
+    return null;
+  }
+
+  public static boolean isApplicationInState(String text, String applicationName) {
+    boolean exists = false
+    SliderShell shell = slider(EXIT_SUCCESS,
+        [
+            ACTION_LIST,
+            applicationName])
+    for (String str in shell.out) {
+      if (str.contains(text)) {
+        exists = true
+      }
+    }
+
+    return exists
+  }
+
+  protected void ensureApplicationIsUp(String clusterName) {
+    repeatUntilTrue(this.&isApplicationUp, 15, 1000 * 3, ['arg1': clusterName],
+        true, 'Application did not start, aborting test.')
+  }
+
+  boolean isApplicationUp(Map<String, String> args) {
+    String applicationName = args['arg1'];
+    return isApplicationInState("RUNNING", applicationName);
+  }
+
+  public static void addDir(File dirObj, ZipOutputStream zipFile, String prefix) {
+    dirObj.eachFile() { file ->
+      if (file.directory) {
+        addDir(file, zipFile, prefix + file.name + File.separator)
+      } else {
+        log.info("Adding to zip - " + prefix + file.getName())
+        zipFile.putNextEntry(new ZipEntry(prefix + file.getName()))
+        file.eachByte(1024) { buffer, len -> zipFile.write(buffer, 0, len) }
+        zipFile.closeEntry()
+      }
+    }
+  }
+
+  protected void repeatUntilTrue(Closure c, int maxAttempts, int sleepDur, Map args,
+                                 boolean failIfUnsuccessful = false, String message = "") {
+    int attemptCount = 0
+    while (attemptCount < maxAttempts) {
+      if (c(args)) {
+        break
+      };
+      attemptCount++;
+
+      if (failIfUnsuccessful) {
+        assert attemptCount != maxAttempts, message
+      }
+
+      sleep(sleepDur)
+    }
+  }
+
+  protected void cleanup(String applicationName) throws Throwable {
+    if (setup_failed) {
+      // cleanup probably won't work if setup failed
+      return
+    }
+
+    log.info "Cleaning app instance, if exists, by name " + applicationName
+    teardown(applicationName)
+
+    // sleep till the instance is frozen
+    sleep(1000 * 3)
+
+    SliderShell shell = slider([
+        ACTION_DESTROY,
+        applicationName])
+
+    if (shell.ret != 0 && shell.ret != EXIT_UNKNOWN_INSTANCE) {
+      logShell(shell)
+      assert fail("Old cluster either should not exist or should get destroyed.")
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/slider-funtest/src/main/java/org/apache/slider/funtest/accumulo/StubToForceGroovySrcToCompile.java
----------------------------------------------------------------------
diff --git a/slider-funtest/src/main/java/org/apache/slider/funtest/accumulo/StubToForceGroovySrcToCompile.java b/slider-funtest/src/main/java/org/apache/slider/funtest/accumulo/StubToForceGroovySrcToCompile.java
new file mode 100644
index 0000000..eefccbb
--- /dev/null
+++ b/slider-funtest/src/main/java/org/apache/slider/funtest/accumulo/StubToForceGroovySrcToCompile.java
@@ -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.
+ */
+
+package org.apache.slider.funtest.accumulo;
+
+class StubToForceGroovySrcToCompile {
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentCommandTestBase.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentCommandTestBase.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentCommandTestBase.groovy
deleted file mode 100644
index daf102a..0000000
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/AgentCommandTestBase.groovy
+++ /dev/null
@@ -1,233 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.slider.funtest.lifecycle
-
-import groovy.util.logging.Slf4j
-import org.apache.hadoop.fs.Path
-import org.apache.slider.common.SliderExitCodes
-import org.apache.slider.common.params.Arguments
-import org.apache.slider.common.params.SliderActions
-import org.apache.slider.funtest.framework.AgentUploads
-import org.apache.slider.funtest.framework.CommandTestBase
-import org.apache.slider.funtest.framework.FuntestProperties
-import org.apache.slider.funtest.framework.SliderShell
-import org.apache.tools.zip.ZipEntry
-import org.apache.tools.zip.ZipOutputStream
-import org.junit.Before
-import org.junit.BeforeClass
-import org.junit.Rule
-import org.junit.rules.TemporaryFolder
-
-@Slf4j
-class AgentCommandTestBase extends CommandTestBase
-implements FuntestProperties, Arguments, SliderExitCodes, SliderActions {
-
-  public static final boolean AGENTTESTS_ENABLED
-  private static String TEST_APP_PKG_PROP = "test.app.pkg"
-
-
-  protected static String APP_RESOURCE = "../slider-core/src/test/app_packages/test_command_log/resources.json"
-  protected static String APP_TEMPLATE = "../slider-core/src/test/app_packages/test_command_log/appConfig.json"
-  protected static String AGENT_CONF = "../slider-agent/conf/agent.ini"
-  protected static final File LOCAL_AGENT_CONF
-  public static final String TEST_APP_PKG = sysprop(TEST_APP_PKG_PROP)
-
-
-  protected static Path agentTarballPath;
-  protected static Path appPkgPath;
-  protected static Path agtIniPath;
-
-  static {
-    AGENTTESTS_ENABLED = SLIDER_CONFIG.getBoolean(KEY_TEST_AGENT_ENABLED, false)
-    LOCAL_AGENT_CONF = new File(AGENT_CONF).canonicalFile
-  }
-
-  @Rule
-  public TemporaryFolder folder = new TemporaryFolder();
-
-  public static void assumeAgentTestsEnabled() {
-    assumeFunctionalTestsEnabled()
-    assume(AGENTTESTS_ENABLED, "Agent tests disabled")
-  }
-
-  @BeforeClass
-  public static void setupAgent() {
-    assumeAgentTestsEnabled()
-
-  }
-  
-  @Before
-  public void uploadAgentTarball() {
-    def agentUploads = new AgentUploads(SLIDER_CONFIG)
-    (agentTarballPath, agtIniPath) = 
-        agentUploads.uploadAgentFiles(SLIDER_TAR_DIRECTORY, false)
-  } 
-
-
-  @Before
-  public void setupApplicationPackage() {
-    AgentUploads agentUploads = new AgentUploads(SLIDER_CONFIG)
-    agentUploads.uploader.mkHomeDir()
-
-    appPkgPath = new Path(clusterFS.homeDirectory, "apache-slider-command-logger.zip")
-    if (!clusterFS.exists(appPkgPath)) {
-      clusterFS.delete(appPkgPath, false)
-    }
-
-    File zipFileName = new File(TEST_APP_PKG).canonicalFile
-    agentUploads.uploader.copyIfOutOfDate(zipFileName, appPkgPath, false)
-  }
-
-  public static void logShell(SliderShell shell) {
-    for (String str in shell.out) {
-      log.info str
-    }
-  }
-
-  public static void assertComponentCount(String component, int count, SliderShell shell) {
-    log.info("Asserting component count.")
-    String entry = findLineEntry(shell, ["instances", component] as String[])
-    log.info(entry)
-    assert entry != null
-    int instanceCount = 0
-    int index = entry.indexOf("container_")
-    while (index != -1) {
-      instanceCount++;
-      index = entry.indexOf("container_", index + 1)
-    }
-
-    assert instanceCount == count, 'Instance count for component did not match expected. Parsed: ' + entry
-  }
-
-  public static String findLineEntry(SliderShell shell, String[] locaters) {
-    int index = 0;
-    for (String str in shell.out) {
-      if (str.contains("\"" + locaters[index] + "\"")) {
-        if (locaters.size() == index + 1) {
-          return str;
-        } else {
-          index++;
-        }
-      }
-    }
-
-    return null;
-  }
-
-  public static String findLineEntryValue(SliderShell shell, String[] locaters) {
-    String line = findLineEntry(shell, locaters);
-
-    if (line != null) {
-      log.info("Parsing {} for value.", line)
-      int dividerIndex = line.indexOf(":");
-      if (dividerIndex > 0) {
-        String value = line.substring(dividerIndex + 1).trim()
-        if (value.endsWith(",")) {
-          value = value.subSequence(0, value.length() - 1)
-        }
-        return value;
-      }
-    }
-    return null;
-  }
-
-  public static boolean isApplicationInState(String text, String applicationName) {
-    boolean exists = false
-    SliderShell shell = slider(EXIT_SUCCESS,
-        [
-            ACTION_LIST,
-            applicationName])
-    for (String str in shell.out) {
-      if (str.contains(text)) {
-        exists = true
-      }
-    }
-
-    return exists
-  }
-
-  protected void ensureApplicationIsUp(String clusterName) {
-    repeatUntilTrue(this.&isApplicationUp, 15, 1000 * 3, ['arg1': clusterName],
-        true, 'Application did not start, aborting test.')
-  }
-
-  boolean isApplicationUp(Map<String, String> args) {
-    String applicationName = args['arg1'];
-    return isApplicationInState("RUNNING", applicationName);
-  }
-
-  public static void addDir(File dirObj, ZipOutputStream zipFile, String prefix) {
-    dirObj.eachFile() { file ->
-      if (file.directory) {
-        addDir(file, zipFile, prefix + file.name + File.separator)
-      } else {
-        log.info("Adding to zip - " + prefix + file.getName())
-        zipFile.putNextEntry(new ZipEntry(prefix + file.getName()))
-        file.eachByte(1024) { buffer, len -> zipFile.write(buffer, 0, len) }
-        zipFile.closeEntry()
-      }
-    }
-  }
-
-  public static void zipDir(String zipFile, String dir) {
-    File dirObj = new File(dir);
-    ZipOutputStream out = new ZipOutputStream(new FileOutputStream(zipFile));
-    log.info("Creating : " + zipFile);
-    try {
-      addDir(dirObj, out, "");
-    }
-    finally {
-      out.close();
-    }
-  }
-
-  protected void repeatUntilTrue(Closure c, int maxAttempts, int sleepDur, Map args,
-                                 boolean failIfUnsuccessful = false, String message = "") {
-    int attemptCount = 0
-    while (attemptCount < maxAttempts) {
-      if (c(args)) {
-        break
-      };
-      attemptCount++;
-
-      if (failIfUnsuccessful) {
-        assert attemptCount != maxAttempts, message
-      }
-
-      sleep(sleepDur)
-    }
-  }
-
-  protected void cleanup(String applicationName) throws Throwable {
-    log.info "Cleaning app instance, if exists, by name " + applicationName
-    teardown(applicationName)
-
-    // sleep till the instance is frozen
-    sleep(1000 * 3)
-
-    SliderShell shell = slider([
-        ACTION_DESTROY,
-        applicationName])
-
-    if (shell.ret != 0 && shell.ret != EXIT_UNKNOWN_INSTANCE) {
-      logShell(shell)
-      assert fail("Old cluster either should not exist or should get destroyed.")
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy
index 25a4134..0d643ca 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentClusterLifecycle.groovy
@@ -27,6 +27,7 @@ import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.SliderXmlConfKeys
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
+import org.apache.slider.funtest.framework.AgentCommandTestBase
 import org.apache.slider.funtest.framework.FuntestProperties
 import org.apache.slider.funtest.framework.SliderShell
 import org.junit.After

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentFailures.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentFailures.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentFailures.groovy
index 96a99ad..ea58d5f 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentFailures.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentFailures.groovy
@@ -23,6 +23,7 @@ import groovy.util.logging.Slf4j
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
+import org.apache.slider.funtest.framework.AgentCommandTestBase
 import org.apache.slider.funtest.framework.FuntestProperties
 import org.apache.slider.funtest.framework.SliderShell
 import org.junit.After

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentFailures2.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentFailures2.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentFailures2.groovy
index 644fa4f..7804042 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentFailures2.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAgentFailures2.groovy
@@ -23,6 +23,7 @@ import groovy.util.logging.Slf4j
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
+import org.apache.slider.funtest.framework.AgentCommandTestBase
 import org.apache.slider.funtest.framework.FuntestProperties
 import org.apache.slider.funtest.framework.SliderShell
 import org.junit.After

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAppsThroughAgent.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAppsThroughAgent.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAppsThroughAgent.groovy
index 793a323..6b0f678 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAppsThroughAgent.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestAppsThroughAgent.groovy
@@ -23,6 +23,7 @@ import groovy.util.logging.Slf4j
 import org.apache.slider.common.SliderExitCodes
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
+import org.apache.slider.funtest.framework.AgentCommandTestBase
 import org.apache.slider.funtest.framework.FuntestProperties
 import org.apache.slider.funtest.framework.SliderShell
 import org.junit.After

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterBuildDestroy.groovy
----------------------------------------------------------------------
diff --git a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterBuildDestroy.groovy b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterBuildDestroy.groovy
index 1907a2c..ead1601 100644
--- a/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterBuildDestroy.groovy
+++ b/slider-funtest/src/test/groovy/org/apache/slider/funtest/lifecycle/TestClusterBuildDestroy.groovy
@@ -26,6 +26,7 @@ import org.apache.slider.common.SliderKeys
 import org.apache.slider.common.SliderXmlConfKeys
 import org.apache.slider.common.params.Arguments
 import org.apache.slider.common.params.SliderActions
+import org.apache.slider.funtest.framework.AgentCommandTestBase
 import org.apache.slider.funtest.framework.FuntestProperties
 import org.junit.AfterClass
 import org.junit.BeforeClass

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/f94e1039/slider-providers/accumulo/slider-accumulo-provider/src/main/java/org/apache/slider/providers/accumulo/AccumuloProviderService.java
----------------------------------------------------------------------
diff --git a/slider-providers/accumulo/slider-accumulo-provider/src/main/java/org/apache/slider/providers/accumulo/AccumuloProviderService.java b/slider-providers/accumulo/slider-accumulo-provider/src/main/java/org/apache/slider/providers/accumulo/AccumuloProviderService.java
index 4f5f2fa..c511efb 100644
--- a/slider-providers/accumulo/slider-accumulo-provider/src/main/java/org/apache/slider/providers/accumulo/AccumuloProviderService.java
+++ b/slider-providers/accumulo/slider-accumulo-provider/src/main/java/org/apache/slider/providers/accumulo/AccumuloProviderService.java
@@ -410,9 +410,13 @@ public class AccumuloProviderService extends AbstractProviderService implements
     String monitorKey = "Active Accumulo Monitor: ";
     String monitorAddr = getInfoAvoidingNull(clusterDesc, AccumuloKeys.MONITOR_ADDRESS);
     if (!StringUtils.isBlank(monitorAddr)) {
+      try {
         HostAndPort hostPort = HostAndPort.fromString(monitorAddr);
         details.put(monitorKey,
             String.format("http://%s:%d", hostPort.getHostText(), hostPort.getPort()));
+      } catch (Exception e) {
+        details.put(monitorKey + "N/A", null);
+      }
     } else {
       details.put(monitorKey + "N/A", null);
     }


[38/50] [abbrv] SLIDER-131 enabling agent to AM one way and two way SSL

Posted by sm...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/server/services/security/SignMessage.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/security/SignMessage.java b/slider-core/src/main/java/org/apache/slider/server/services/security/SignMessage.java
new file mode 100644
index 0000000..4bccb87
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/services/security/SignMessage.java
@@ -0,0 +1,54 @@
+/**
+ * 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.server.services.security;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ *
+ * Sign certificate request data model.
+ *
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlType(name = "", propOrder = {})
+public class SignMessage {
+
+  @XmlElement
+  private String csr;
+  @XmlElement
+  private String passphrase;
+  public String getCsr() {
+    return csr;
+  }
+  public void setCsr(String csr) {
+    this.csr = csr;
+  }
+  public String getPassphrase() {
+    return passphrase;
+  }
+  public void setPassphrase(String passphrase) {
+    this.passphrase = passphrase;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/appconf.json
----------------------------------------------------------------------
diff --git a/slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/appconf.json b/slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/appconf.json
index f28159a..89095b1 100644
--- a/slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/appconf.json
+++ b/slider-core/src/main/resources/org/apache/slider/providers/slideram/instance/appconf.json
@@ -12,8 +12,9 @@
 
   "components": {
     "slider-appmaster" : {
-      "jvm.heapsize": "256M"
+      "jvm.heapsize": "256M",
+      "ssl.server.keystore.location": "/tmp/work/security/keystore.p12"
     }
 
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/resources/webapps/slideragent/.keep
----------------------------------------------------------------------
diff --git a/slider-core/src/main/resources/webapps/slideragent/.keep b/slider-core/src/main/resources/webapps/slideragent/.keep
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy b/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy
index 51d9bd6..7d68458 100644
--- a/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/providers/agent/TestAgentAMManagementWS.groovy
@@ -24,11 +24,17 @@ import groovy.transform.CompileStatic
 import groovy.util.logging.Slf4j
 import org.apache.slider.api.StatusKeys
 import org.apache.slider.client.SliderClient
+import org.apache.slider.common.SliderKeys
+import org.apache.slider.core.conf.MapOperations
 import org.apache.slider.core.main.ServiceLauncher
 import org.apache.slider.server.appmaster.web.SliderAMWebApp
 import org.apache.slider.server.appmaster.web.rest.agent.RegistrationResponse
 import org.apache.slider.server.appmaster.web.rest.agent.RegistrationStatus
+import org.apache.slider.server.services.security.CertificateManager
+import org.apache.slider.server.services.security.SecurityUtils
 import org.junit.Test
+import org.slf4j.Logger
+import org.slf4j.LoggerFactory
 
 import javax.ws.rs.core.MediaType
 
@@ -42,8 +48,42 @@ import static org.apache.slider.test.SliderTestUtils.log
 @Slf4j
 class TestAgentAMManagementWS extends AgentTestBase {
 
-  public static final String MANAGEMENT_URI = SliderAMWebApp.BASE_PATH +"/ws/v1/slider/mgmt/";
   public static final String AGENT_URI = "ws/v1/slider/agents/";
+    final static Logger logger = LoggerFactory.getLogger(TestAgentAMManagementWS.class)
+    static {
+        //for localhost testing only
+        javax.net.ssl.HttpsURLConnection.setDefaultHostnameVerifier(
+                new javax.net.ssl.HostnameVerifier(){
+                    public boolean verify(String hostname,
+                                          javax.net.ssl.SSLSession sslSession) {
+                        logger.info("verifying hostname ${hostname}")
+                        InetAddress[] addresses =
+                            InetAddress.getAllByName(hostname);
+                        if (hostname.equals("localhost")) {
+                            return true;
+                        }
+                        for (InetAddress address : addresses) {
+                            if (address.getHostName().equals(hostname) ||
+                                address.isAnyLocalAddress() ||
+                                address.isLoopbackAddress()) {
+                                return true;
+                            }
+                        }
+                        return false;
+                    }
+                });
+
+        MapOperations compOperations = new MapOperations();
+        compOperations.put(SliderKeys.KEYSTORE_LOCATION, "/tmp/work/security/keystore.p12");
+        SecurityUtils.initializeSecurityParameters(compOperations);
+        CertificateManager certificateManager = new CertificateManager();
+        certificateManager.initRootCert(compOperations);
+        String keystoreFile = SecurityUtils.getSecurityDir() + File.separator + SliderKeys.KEYSTORE_FILE_NAME;
+        String password = SecurityUtils.getKeystorePass();
+        System.setProperty("javax.net.ssl.trustStore", keystoreFile);
+        System.setProperty("javax.net.ssl.trustStorePassword", password);
+        System.setProperty("javax.net.ssl.trustStoreType", "PKCS12");
+    }
 
   @Test
   public void testAgentAMManagementWS() throws Throwable {
@@ -84,7 +124,7 @@ class TestAgentAMManagementWS extends AgentTestBase {
 
     
     def status = dumpClusterStatus(sliderClient, "agent AM")
-    def liveURL = status.getInfo(StatusKeys.INFO_AM_WEB_URL) 
+    def liveURL = status.getInfo(StatusKeys.INFO_AM_AGENT_URL)
     if (liveURL) {
       agent_url = liveURL + AGENT_URI
     }
@@ -94,7 +134,7 @@ class TestAgentAMManagementWS extends AgentTestBase {
     log.info("conf   is ${liveURL}conf")
 
 
-    def sleeptime = 60
+    def sleeptime = 10
     log.info "sleeping for $sleeptime seconds"
     Thread.sleep(sleeptime * 1000)
     

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy
index 873bc93..7b73451 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/model/mock/MockProviderService.groovy
@@ -21,7 +21,6 @@ package org.apache.slider.server.appmaster.model.mock
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.service.LifecycleEvent
-import org.apache.hadoop.service.Service.STATE
 import org.apache.hadoop.service.ServiceStateChangeListener
 import org.apache.hadoop.yarn.api.records.Container
 import org.apache.slider.api.ClusterDescription
@@ -44,6 +43,7 @@ import org.apache.slider.server.appmaster.web.rest.agent.RegistrationResponse
 import org.apache.slider.server.appmaster.web.rest.agent.RegistrationStatus
 import org.apache.slider.server.services.registry.RegistryViewForProviders
 import org.apache.slider.providers.ProviderCompleted
+import org.apache.hadoop.service.Service.STATE
 
 class MockProviderService implements ProviderService {
 
@@ -95,12 +95,11 @@ class MockProviderService implements ProviderService {
     return null;
   }
 
-  @Override
   public STATE getServiceState() {
-    return null;
+    return null
   }
 
-  @Override
+    @Override
   public long getStartTime() {
     return 0;
   }
@@ -188,7 +187,7 @@ class MockProviderService implements ProviderService {
   }
 
   @Override
-  public Map<String, URL> buildMonitorDetails(ClusterDescription clusterSpec) {
+  public Map<String, String> buildMonitorDetails(ClusterDescription clusterSpec) {
     return null;
   }
 
@@ -226,8 +225,7 @@ class MockProviderService implements ProviderService {
 
   @Override
   void applyInitialRegistryDefinitions(
-      URL amWebAPI,
-      ServiceInstanceData registryInstanceData)
+          URL unsecureWebAPI, URL secureWebAPI, ServiceInstanceData registryInstanceData)
   throws MalformedURLException, IOException {
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy
index 4304452..1c98e94 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestClusterSpecificationBlock.groovy
@@ -52,7 +52,7 @@ public class TestClusterSpecificationBlock {
         appState)
     ProviderService providerService = new MockProviderService();
 
-    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, providerAppState, providerService);
+    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, providerAppState, providerService, null);
 
     Injector injector = Guice.createInjector(new AbstractModule() {
           @Override

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy
index 69cdd2b..f871b07 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestContainerStatsBlock.groovy
@@ -61,7 +61,7 @@ public class TestContainerStatsBlock {
         "undefined",
         appState)
 
-    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, providerAppState, providerService);
+    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, providerAppState, providerService, null);
 
     Injector injector = Guice.createInjector(new AbstractModule() {
           @Override

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
index 176299d..6b46591 100644
--- a/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/server/appmaster/web/view/TestIndexBlock.groovy
@@ -51,7 +51,7 @@ public class TestIndexBlock {
         "undefined",
         appState)
 
-    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, providerAppState, providerService);
+    WebAppApiImpl inst = new WebAppApiImpl(clusterProto, providerAppState, providerService, null);
 
     Injector injector = Guice.createInjector(new AbstractModule() {
           @Override

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/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 9e9e0db..c10b60a 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
@@ -255,7 +255,8 @@ public class TestAgentProviderService {
     expect(access.isApplicationLive()).andReturn(true).anyTimes();
     ClusterDescription desc = new ClusterDescription();
     desc.setInfo(StatusKeys.INFO_AM_HOSTNAME, "host1");
-    desc.setInfo(StatusKeys.INFO_AM_WEB_PORT, "8088");
+    desc.setInfo(StatusKeys.INFO_AM_AGENT_PORT, "8088");
+    desc.setInfo(StatusKeys.INFO_AM_SECURED_AGENT_PORT, "8089");
     desc.setInfo(OptionKeys.APPLICATION_NAME, "HBASE");
     expect(access.getClusterStatus()).andReturn(desc).anyTimes();
 
@@ -578,7 +579,8 @@ public class TestAgentProviderService {
     expect(access.isApplicationLive()).andReturn(true).anyTimes();
     ClusterDescription desc = new ClusterDescription();
     desc.setInfo(StatusKeys.INFO_AM_HOSTNAME, "host1");
-    desc.setInfo(StatusKeys.INFO_AM_WEB_PORT, "8088");
+    desc.setInfo(StatusKeys.INFO_AM_AGENT_PORT, "8088");
+    desc.setInfo(StatusKeys.INFO_AM_SECURED_AGENT_PORT, "8089");
     desc.setInfo(OptionKeys.APPLICATION_NAME, "HBASE");
     expect(access.getClusterStatus()).andReturn(desc).anyTimes();
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
index ab953b8..17fbe2b 100644
--- a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
+++ b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/agent/TestAMAgentWebServices.java
@@ -18,26 +18,20 @@
 
 package org.apache.slider.server.appmaster.web.rest.agent;
 
-import com.google.inject.Guice;
-import com.google.inject.Inject;
-import com.google.inject.Injector;
-import com.google.inject.servlet.GuiceServletContextListener;
-import com.google.inject.servlet.ServletModule;
 import com.sun.jersey.api.client.Client;
 import com.sun.jersey.api.client.ClientResponse;
 import com.sun.jersey.api.client.WebResource;
 import com.sun.jersey.api.client.config.ClientConfig;
 import com.sun.jersey.api.client.config.DefaultClientConfig;
 import com.sun.jersey.api.json.JSONConfiguration;
-import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
 import com.sun.jersey.test.framework.JerseyTest;
-import com.sun.jersey.test.framework.WebAppDescriptor;
 import junit.framework.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.slider.common.SliderKeys;
 import org.apache.slider.common.tools.SliderUtils;
+import org.apache.slider.core.conf.MapOperations;
 import org.apache.slider.server.appmaster.model.mock.MockFactory;
 import org.apache.slider.server.appmaster.model.mock.MockProviderService;
 import org.apache.slider.server.appmaster.model.mock.MockRecordFactory;
@@ -46,135 +40,130 @@ import org.apache.slider.server.appmaster.state.AppState;
 import org.apache.slider.server.appmaster.state.ProviderAppState;
 import org.apache.slider.server.appmaster.web.WebAppApi;
 import org.apache.slider.server.appmaster.web.WebAppApiImpl;
-import org.apache.slider.server.appmaster.web.rest.AMWebServices;
-import org.apache.slider.server.appmaster.web.rest.SliderJacksonJaxbJsonProvider;
+import org.apache.slider.server.appmaster.web.rest.RestPaths;
+import org.apache.slider.server.services.security.CertificateManager;
+import org.apache.slider.server.services.security.SecurityUtils;
+import org.apache.slider.test.SliderTestBase;
+import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.ws.rs.Path;
 import javax.ws.rs.core.MediaType;
 import java.io.File;
+import java.io.IOException;
 import java.net.URI;
-import java.util.HashMap;
-import java.util.Map;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
 
 import static org.junit.Assert.assertEquals;
 
-public class TestAMAgentWebServices extends JerseyTest {
+public class TestAMAgentWebServices {
+
+  static {
+    //for localhost testing only
+    javax.net.ssl.HttpsURLConnection.setDefaultHostnameVerifier(
+        new javax.net.ssl.HostnameVerifier(){
+
+          public boolean verify(String hostname,
+                                javax.net.ssl.SSLSession sslSession) {
+            if (hostname.equals("localhost")) {
+              return true;
+            }
+            return false;
+          }
+        });
+
+    SecurityUtils.initializeSecurityParameters(new MapOperations());
+    MapOperations compOperations = new MapOperations();
+    CertificateManager certificateManager = new CertificateManager();
+    certificateManager.initRootCert(compOperations);
+    String keystoreFile = SecurityUtils.getSecurityDir() + File.separator + SliderKeys.KEYSTORE_FILE_NAME;
+    String password = SecurityUtils.getKeystorePass();
+    System.setProperty("javax.net.ssl.trustStore", keystoreFile);
+    System.setProperty("javax.net.ssl.trustStorePassword", password);
+    System.setProperty("javax.net.ssl.trustStoreType", "PKCS12");
+  }
+
   protected static final Logger log =
     LoggerFactory.getLogger(TestAMAgentWebServices.class);
   
   public static final int RM_MAX_RAM = 4096;
   public static final int RM_MAX_CORES = 64;
   public static final String AGENT_URL =
-    "http://localhost:9998/slideram/ws/v1/slider/agents/";
+    "https://localhost:${PORT}/ws/v1/slider/agents/";
   
   static MockFactory factory = new MockFactory();
   private static Configuration conf = new Configuration();
   private static WebAppApi slider;
 
-  private static Injector injector = createInjector();
   private static FileSystem fs;
-
-  public static class GuiceServletConfig extends GuiceServletContextListener {
-
-    public GuiceServletConfig() {
-      super();
-    }
-
-    @Override
-    protected Injector getInjector() {
-      return injector;
-    }
-  }
-
-//  @Path("/ws/v1/slider/agent")
-  @Path("/ws/v1/slider")
-  public static class MockAMWebServices extends AMWebServices {
-
-    @Inject
-    public MockAMWebServices(WebAppApi slider) {
-      super(slider);
-    }
-
-  }
+  private AgentWebApp webApp;
+  private String base_url;
 
   @Before
-  @Override
   public void setUp() throws Exception {
-    super.setUp();
-    injector = createInjector();
     YarnConfiguration conf = SliderUtils.createConfiguration();
     fs = FileSystem.get(new URI("file:///"), conf);
-  }
+    AppState appState = null;
+    try {
+      fs = FileSystem.get(new URI("file:///"), conf);
+      File
+          historyWorkDir =
+          new File("target/history", "TestAMAgentWebServices");
+      org.apache.hadoop.fs.Path
+          historyPath =
+          new org.apache.hadoop.fs.Path(historyWorkDir.toURI());
+      fs.delete(historyPath, true);
+      appState = new AppState(new MockRecordFactory());
+      appState.setContainerLimits(RM_MAX_RAM, RM_MAX_CORES);
+      appState.buildInstance(
+          factory.newInstanceDefinition(0, 0, 0),
+          new Configuration(false),
+          factory.ROLES,
+          fs,
+          historyPath,
+          null, null);
+    } catch (Exception e) {
+      log.error("Failed to set up app {}", e);
+    }
+    ProviderAppState providerAppState = new ProviderAppState("undefined",
+                                                             appState);
+
+    slider = new WebAppApiImpl(new MockSliderClusterProtocol(), providerAppState,
+                               new MockProviderService(), null);
+
+    MapOperations compOperations = new MapOperations();
+
+    webApp = AgentWebApp.$for(AgentWebApp.BASE_PATH, slider,
+                              RestPaths.WS_AGENT_CONTEXT_ROOT)
+        .withComponentConfig(compOperations)
+        .start();
+    base_url = AGENT_URL.replace("${PORT}",
+                                 Integer.toString(webApp.getSecuredPort()));
 
-  private static Injector createInjector() {
-    return Guice.createInjector(new ServletModule() {
-      @Override
-      protected void configureServlets() {
-
-        AppState appState = null;
-        try {
-          fs = FileSystem.get(new URI("file:///"), conf);
-          File
-              historyWorkDir =
-              new File("target/history", "TestAMAgentWebServices");
-          org.apache.hadoop.fs.Path
-              historyPath =
-              new org.apache.hadoop.fs.Path(historyWorkDir.toURI());
-          fs.delete(historyPath, true);
-          appState = new AppState(new MockRecordFactory());
-          appState.setContainerLimits(RM_MAX_RAM, RM_MAX_CORES);
-          appState.buildInstance(
-              factory.newInstanceDefinition(0, 0, 0),
-              new Configuration(false),
-              factory.ROLES,
-              fs,
-              historyPath,
-              null, null);
-        } catch (Exception e) {
-          log.error("Failed to set up app {}", e);
-        }
-        ProviderAppState providerAppState = new ProviderAppState("undefined",
-            appState);
-
-        slider = new WebAppApiImpl(new MockSliderClusterProtocol(), providerAppState,
-                                   new MockProviderService());
-
-        bind(SliderJacksonJaxbJsonProvider.class);
-        bind(GenericExceptionHandler.class);
-        bind(MockAMWebServices.class);
-        bind(WebAppApi.class).toInstance(slider);
-        bind(Configuration.class).toInstance(conf);
-
-        Map<String, String> params = new HashMap<String, String>();
-        addLoggingFilter(params);
-        serve("/*").with(GuiceContainer.class, params);
-      }
-    });
   }
 
-  private static void addLoggingFilter(Map<String, String> params) {
-    params.put("com.sun.jersey.spi.container.ContainerRequestFilters", "com.sun.jersey.api.container.filter.LoggingFilter");
-    params.put("com.sun.jersey.spi.container.ContainerResponseFilters", "com.sun.jersey.api.container.filter.LoggingFilter");
+  @After
+  public void tearDown () throws Exception {
+    webApp.stop();
+    webApp = null;
   }
 
   public TestAMAgentWebServices() {
-    super(new WebAppDescriptor.Builder(
-      "org.apache.hadoop.yarn.appmaster.web")
-            .contextListenerClass(GuiceServletConfig.class)
-            .filterClass(com.google.inject.servlet.GuiceFilter.class)
-            .initParam("com.sun.jersey.api.json.POJOMappingFeature", "true")
-            .contextPath("slideram").servletPath("/").build());
   }
 
   @Test
   public void testRegistration() throws Exception {
     RegistrationResponse response;
     Client client = createTestClient();
-    WebResource webResource = client.resource(AGENT_URL + "test/register");
+    WebResource webResource = client.resource(base_url + "test/register");
     response = webResource.type(MediaType.APPLICATION_JSON)
         .post(RegistrationResponse.class, createDummyJSONRegister());
     Assert.assertEquals(RegistrationStatus.OK, response.getResponseStatus());
@@ -190,7 +179,7 @@ public class TestAMAgentWebServices extends JerseyTest {
   public void testHeartbeat() throws Exception {
     HeartBeatResponse response;
     Client client = createTestClient();
-    WebResource webResource = client.resource(AGENT_URL + "test/heartbeat");
+    WebResource webResource = client.resource(base_url + "test/heartbeat");
     response = webResource.type(MediaType.APPLICATION_JSON)
         .post(HeartBeatResponse.class, createDummyHeartBeat());
     assertEquals(response.getResponseId(), 0L);
@@ -199,17 +188,17 @@ public class TestAMAgentWebServices extends JerseyTest {
   @Test
   public void testHeadURL() throws Exception {
     Client client = createTestClient();
-    WebResource webResource = client.resource(AGENT_URL);
+    WebResource webResource = client.resource(base_url);
     ClientResponse response = webResource.type(MediaType.APPLICATION_JSON)
                                          .head();
     assertEquals(200, response.getStatus());
   }
 
-  @Test
-  public void testSleepForAWhile() throws Throwable {
-    log.info("Agent is running at {}", AGENT_URL);
-    Thread.sleep(60 * 1000);
-  }
+//  @Test
+//  public void testSleepForAWhile() throws Throwable {
+//    log.info("Agent is running at {}", base_url);
+//    Thread.sleep(60 * 1000);
+//  }
   
   private Register createDummyJSONRegister() {
     Register register = new Register();
@@ -227,4 +216,24 @@ public class TestAMAgentWebServices extends JerseyTest {
     return json;
   }
 
+  @AfterClass
+  public static void tearDownClass() throws Exception{
+    Path directory = Paths.get(SecurityUtils.getSecurityDir());
+    Files.walkFileTree(directory, new SimpleFileVisitor<Path>() {
+      @Override
+      public FileVisitResult visitFile(Path file, BasicFileAttributes attrs)
+          throws IOException {
+        Files.delete(file);
+        return FileVisitResult.CONTINUE;
+      }
+
+      @Override
+      public FileVisitResult postVisitDirectory(Path dir, IOException exc)
+          throws IOException {
+        Files.delete(dir);
+        return FileVisitResult.CONTINUE;
+      }
+
+    });
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
index 5cb31a9..134f0bc 100644
--- a/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
+++ b/slider-core/src/test/java/org/apache/slider/server/appmaster/web/rest/management/TestAMManagementWebServices.java
@@ -182,7 +182,7 @@ public class TestAMManagementWebServices extends JerseyTest {
             appState);
 
         slider = new WebAppApiImpl(new MockSliderClusterProtocol(), providerAppState,
-                                   new MockProviderService());
+                                   new MockProviderService(), null);
 
         bind(SliderJacksonJaxbJsonProvider.class);
         bind(MockSliderAMWebServices.class);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/test/java/org/apache/slider/server/services/security/TestCertificateManager.java
----------------------------------------------------------------------
diff --git a/slider-core/src/test/java/org/apache/slider/server/services/security/TestCertificateManager.java b/slider-core/src/test/java/org/apache/slider/server/services/security/TestCertificateManager.java
new file mode 100644
index 0000000..6d2d051
--- /dev/null
+++ b/slider-core/src/test/java/org/apache/slider/server/services/security/TestCertificateManager.java
@@ -0,0 +1,60 @@
+/*
+ * 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.server.services.security;
+
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.core.conf.MapOperations;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+
+/**
+ *
+ */
+public class TestCertificateManager {
+  @Rule
+  public TemporaryFolder workDir = new TemporaryFolder();
+  private File secDir;
+
+  @Before
+  public void setup() throws Exception {
+    CertificateManager certMan = new CertificateManager();
+    MapOperations compOperations = new MapOperations();
+    secDir = new File(workDir.getRoot(), SliderKeys.SECURITY_DIR);
+    File keystoreFile = new File(secDir, SliderKeys.KEYSTORE_FILE_NAME);
+    compOperations.put(SliderKeys.KEYSTORE_LOCATION,
+                       keystoreFile.getAbsolutePath());
+    certMan.initRootCert(compOperations);
+  }
+
+  @Test
+  public void testServerCertificateGenerated() throws Exception {
+    File serverCrt = new File(secDir, SliderKeys.CRT_FILE_NAME);
+    Assert.assertTrue(serverCrt.exists());
+  }
+
+  @Test
+  public void testKeystoreGenerated() throws Exception {
+    File keystore = new File(secDir, SliderKeys.KEYSTORE_FILE_NAME);
+    Assert.assertTrue(keystore.exists());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/test/python/agent/main.py
----------------------------------------------------------------------
diff --git a/slider-core/src/test/python/agent/main.py b/slider-core/src/test/python/agent/main.py
index 573b9a7..8b7044e 100755
--- a/slider-core/src/test/python/agent/main.py
+++ b/slider-core/src/test/python/agent/main.py
@@ -36,6 +36,8 @@ def main():
   parser.add_option('--label', dest='label', help='label')
   parser.add_option('--host', dest='host', help='port')
   parser.add_option('--port', dest='port', help='host')
+  parser.add_option('--secured_port', dest='secured_port', help='host')
+
   (options, args) = parser.parse_args()
 
   if options.log_folder:

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java b/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java
index a578441..f9a5628 100644
--- a/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java
+++ b/slider-providers/hbase/slider-hbase-provider/src/main/java/org/apache/slider/providers/hbase/HBaseProviderService.java
@@ -205,9 +205,10 @@ public class HBaseProviderService extends AbstractProviderService implements
 
   @Override
   public void applyInitialRegistryDefinitions(URL web,
-      ServiceInstanceData instanceData) throws
+                                              URL secureWebAPI,
+                                              ServiceInstanceData instanceData) throws
       IOException {
-    super.applyInitialRegistryDefinitions(web, instanceData);
+    super.applyInitialRegistryDefinitions(web, secureWebAPI, instanceData);
   }
 
   @Override


[46/50] [abbrv] git commit: SLIDER-107 updating release POMs for 0.40

Posted by sm...@apache.org.
SLIDER-107 updating release POMs for 0.40


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

Branch: refs/heads/develop
Commit: e09a6a84e400d7648ee224d83490bb1d17f58652
Parents: bedf1c1
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Thu Jul 10 20:26:21 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Thu Jul 10 20:26:21 2014 -0700

----------------------------------------------------------------------
 app-packages/accumulo/pom.xml                              | 2 +-
 app-packages/command-logger/application-pkg/pom.xml        | 2 +-
 app-packages/command-logger/slider-pkg/pom.xml             | 2 +-
 pom.xml                                                    | 2 +-
 slider-agent/pom.xml                                       | 2 +-
 slider-assembly/pom.xml                                    | 2 +-
 slider-core/pom.xml                                        | 2 +-
 slider-funtest/pom.xml                                     | 2 +-
 slider-install/pom.xml                                     | 2 +-
 slider-providers/accumulo/accumulo-funtests/pom.xml        | 2 +-
 slider-providers/accumulo/slider-accumulo-provider/pom.xml | 2 +-
 slider-providers/hbase/hbase-funtests/pom.xml              | 2 +-
 slider-providers/hbase/slider-hbase-provider/pom.xml       | 2 +-
 13 files changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/app-packages/accumulo/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/pom.xml b/app-packages/accumulo/pom.xml
index d23933b..45dfd87 100644
--- a/app-packages/accumulo/pom.xml
+++ b/app-packages/accumulo/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/app-packages/command-logger/application-pkg/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/application-pkg/pom.xml b/app-packages/command-logger/application-pkg/pom.xml
index 7da7259..53f7fd2 100644
--- a/app-packages/command-logger/application-pkg/pom.xml
+++ b/app-packages/command-logger/application-pkg/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
     <relativePath>../../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/app-packages/command-logger/slider-pkg/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/pom.xml b/app-packages/command-logger/slider-pkg/pom.xml
index 42ec8da..0971868 100644
--- a/app-packages/command-logger/slider-pkg/pom.xml
+++ b/app-packages/command-logger/slider-pkg/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
     <relativePath>../../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a4489bb..a9a90ab 100644
--- a/pom.xml
+++ b/pom.xml
@@ -19,7 +19,7 @@
   <groupId>org.apache.slider</groupId>
   <artifactId>slider</artifactId>
   <name>Slider</name>
-  <version>0.31.0-incubating-SNAPSHOT</version>
+  <version>0.40</version>
   <packaging>pom</packaging>
 
   <description>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-agent/pom.xml
----------------------------------------------------------------------
diff --git a/slider-agent/pom.xml b/slider-agent/pom.xml
index efc61ef..7a3b447 100644
--- a/slider-agent/pom.xml
+++ b/slider-agent/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>slider-agent</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/slider-assembly/pom.xml b/slider-assembly/pom.xml
index 2a312bb..0ec91dd 100644
--- a/slider-assembly/pom.xml
+++ b/slider-assembly/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
   </parent>
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-core/pom.xml
----------------------------------------------------------------------
diff --git a/slider-core/pom.xml b/slider-core/pom.xml
index b6120fb..95a7e71 100644
--- a/slider-core/pom.xml
+++ b/slider-core/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
   </parent>
 
   <build>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-funtest/pom.xml
----------------------------------------------------------------------
diff --git a/slider-funtest/pom.xml b/slider-funtest/pom.xml
index 4b2edc5..cb16669 100644
--- a/slider-funtest/pom.xml
+++ b/slider-funtest/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
   </parent>
   <properties>
     <work.dir>package-tmp</work.dir>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-install/pom.xml
----------------------------------------------------------------------
diff --git a/slider-install/pom.xml b/slider-install/pom.xml
index df061a7..b08895f 100644
--- a/slider-install/pom.xml
+++ b/slider-install/pom.xml
@@ -28,7 +28,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-providers/accumulo/accumulo-funtests/pom.xml
----------------------------------------------------------------------
diff --git a/slider-providers/accumulo/accumulo-funtests/pom.xml b/slider-providers/accumulo/accumulo-funtests/pom.xml
index 27d9e89..1b25b33 100644
--- a/slider-providers/accumulo/accumulo-funtests/pom.xml
+++ b/slider-providers/accumulo/accumulo-funtests/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
     <relativePath>../../../</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-providers/accumulo/slider-accumulo-provider/pom.xml
----------------------------------------------------------------------
diff --git a/slider-providers/accumulo/slider-accumulo-provider/pom.xml b/slider-providers/accumulo/slider-accumulo-provider/pom.xml
index 00d2cce..cabea00 100644
--- a/slider-providers/accumulo/slider-accumulo-provider/pom.xml
+++ b/slider-providers/accumulo/slider-accumulo-provider/pom.xml
@@ -28,7 +28,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
     <relativePath>../../../</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-providers/hbase/hbase-funtests/pom.xml
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/hbase-funtests/pom.xml b/slider-providers/hbase/hbase-funtests/pom.xml
index cfebfc2..f5a155b 100644
--- a/slider-providers/hbase/hbase-funtests/pom.xml
+++ b/slider-providers/hbase/hbase-funtests/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
     <relativePath>../../../</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/e09a6a84/slider-providers/hbase/slider-hbase-provider/pom.xml
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/slider-hbase-provider/pom.xml b/slider-providers/hbase/slider-hbase-provider/pom.xml
index 1cd5e66..381a1b6 100644
--- a/slider-providers/hbase/slider-hbase-provider/pom.xml
+++ b/slider-providers/hbase/slider-hbase-provider/pom.xml
@@ -29,7 +29,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.31.0-incubating-SNAPSHOT</version>
+    <version>0.40</version>
     <relativePath>../../../</relativePath>
   </parent>
 


[50/50] [abbrv] git commit: SLIDER-107 updating development POMs to 0.41.0-SNAPSHOT

Posted by sm...@apache.org.
SLIDER-107 updating development POMs to 0.41.0-SNAPSHOT


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

Branch: refs/heads/develop
Commit: 166d25230c76dca1904b440cee3d54497d654c5f
Parents: 454f5cc
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Thu Jul 10 21:17:14 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Thu Jul 10 21:17:14 2014 -0700

----------------------------------------------------------------------
 app-packages/accumulo/pom.xml                              | 2 +-
 app-packages/command-logger/application-pkg/pom.xml        | 2 +-
 app-packages/command-logger/slider-pkg/pom.xml             | 2 +-
 pom.xml                                                    | 2 +-
 slider-agent/pom.xml                                       | 2 +-
 slider-assembly/pom.xml                                    | 2 +-
 slider-core/pom.xml                                        | 2 +-
 slider-funtest/pom.xml                                     | 2 +-
 slider-install/pom.xml                                     | 2 +-
 slider-providers/accumulo/accumulo-funtests/pom.xml        | 2 +-
 slider-providers/accumulo/slider-accumulo-provider/pom.xml | 2 +-
 slider-providers/hbase/hbase-funtests/pom.xml              | 2 +-
 slider-providers/hbase/slider-hbase-provider/pom.xml       | 2 +-
 13 files changed, 13 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/166d2523/app-packages/accumulo/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/accumulo/pom.xml b/app-packages/accumulo/pom.xml
index 45dfd87..985773a 100644
--- a/app-packages/accumulo/pom.xml
+++ b/app-packages/accumulo/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.40</version>
+    <version>0.41.0-SNAPSHOT</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/166d2523/app-packages/command-logger/application-pkg/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/application-pkg/pom.xml b/app-packages/command-logger/application-pkg/pom.xml
index 53f7fd2..100065a 100644
--- a/app-packages/command-logger/application-pkg/pom.xml
+++ b/app-packages/command-logger/application-pkg/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.40</version>
+    <version>0.41.0-SNAPSHOT</version>
     <relativePath>../../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/166d2523/app-packages/command-logger/slider-pkg/pom.xml
----------------------------------------------------------------------
diff --git a/app-packages/command-logger/slider-pkg/pom.xml b/app-packages/command-logger/slider-pkg/pom.xml
index 0971868..a8dcb12 100644
--- a/app-packages/command-logger/slider-pkg/pom.xml
+++ b/app-packages/command-logger/slider-pkg/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.40</version>
+    <version>0.41.0-SNAPSHOT</version>
     <relativePath>../../../pom.xml</relativePath>
   </parent>
   <modelVersion>4.0.0</modelVersion>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/166d2523/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a9a90ab..9ae2a5e 100644
--- a/pom.xml
+++ b/pom.xml
@@ -19,7 +19,7 @@
   <groupId>org.apache.slider</groupId>
   <artifactId>slider</artifactId>
   <name>Slider</name>
-  <version>0.40</version>
+  <version>0.41.0-SNAPSHOT</version>
   <packaging>pom</packaging>
 
   <description>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/166d2523/slider-agent/pom.xml
----------------------------------------------------------------------
diff --git a/slider-agent/pom.xml b/slider-agent/pom.xml
index 7a3b447..21930b8 100644
--- a/slider-agent/pom.xml
+++ b/slider-agent/pom.xml
@@ -19,7 +19,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.40</version>
+    <version>0.41.0-SNAPSHOT</version>
   </parent>
   <modelVersion>4.0.0</modelVersion>
   <artifactId>slider-agent</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/166d2523/slider-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/slider-assembly/pom.xml b/slider-assembly/pom.xml
index 0ec91dd..20ffe69 100644
--- a/slider-assembly/pom.xml
+++ b/slider-assembly/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.40</version>
+    <version>0.41.0-SNAPSHOT</version>
   </parent>
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/166d2523/slider-core/pom.xml
----------------------------------------------------------------------
diff --git a/slider-core/pom.xml b/slider-core/pom.xml
index 95a7e71..f9969fe 100644
--- a/slider-core/pom.xml
+++ b/slider-core/pom.xml
@@ -23,7 +23,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.40</version>
+    <version>0.41.0-SNAPSHOT</version>
   </parent>
 
   <build>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/166d2523/slider-funtest/pom.xml
----------------------------------------------------------------------
diff --git a/slider-funtest/pom.xml b/slider-funtest/pom.xml
index cb16669..5117875 100644
--- a/slider-funtest/pom.xml
+++ b/slider-funtest/pom.xml
@@ -25,7 +25,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.40</version>
+    <version>0.41.0-SNAPSHOT</version>
   </parent>
   <properties>
     <work.dir>package-tmp</work.dir>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/166d2523/slider-install/pom.xml
----------------------------------------------------------------------
diff --git a/slider-install/pom.xml b/slider-install/pom.xml
index b08895f..a2ccc3a 100644
--- a/slider-install/pom.xml
+++ b/slider-install/pom.xml
@@ -28,7 +28,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.40</version>
+    <version>0.41.0-SNAPSHOT</version>
   </parent>
 
   <properties>

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/166d2523/slider-providers/accumulo/accumulo-funtests/pom.xml
----------------------------------------------------------------------
diff --git a/slider-providers/accumulo/accumulo-funtests/pom.xml b/slider-providers/accumulo/accumulo-funtests/pom.xml
index 1b25b33..c55c0d5 100644
--- a/slider-providers/accumulo/accumulo-funtests/pom.xml
+++ b/slider-providers/accumulo/accumulo-funtests/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.40</version>
+    <version>0.41.0-SNAPSHOT</version>
     <relativePath>../../../</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/166d2523/slider-providers/accumulo/slider-accumulo-provider/pom.xml
----------------------------------------------------------------------
diff --git a/slider-providers/accumulo/slider-accumulo-provider/pom.xml b/slider-providers/accumulo/slider-accumulo-provider/pom.xml
index cabea00..c7d6515 100644
--- a/slider-providers/accumulo/slider-accumulo-provider/pom.xml
+++ b/slider-providers/accumulo/slider-accumulo-provider/pom.xml
@@ -28,7 +28,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.40</version>
+    <version>0.41.0-SNAPSHOT</version>
     <relativePath>../../../</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/166d2523/slider-providers/hbase/hbase-funtests/pom.xml
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/hbase-funtests/pom.xml b/slider-providers/hbase/hbase-funtests/pom.xml
index f5a155b..7913fbf 100644
--- a/slider-providers/hbase/hbase-funtests/pom.xml
+++ b/slider-providers/hbase/hbase-funtests/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.40</version>
+    <version>0.41.0-SNAPSHOT</version>
     <relativePath>../../../</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/166d2523/slider-providers/hbase/slider-hbase-provider/pom.xml
----------------------------------------------------------------------
diff --git a/slider-providers/hbase/slider-hbase-provider/pom.xml b/slider-providers/hbase/slider-hbase-provider/pom.xml
index 381a1b6..bf53cd9 100644
--- a/slider-providers/hbase/slider-hbase-provider/pom.xml
+++ b/slider-providers/hbase/slider-hbase-provider/pom.xml
@@ -29,7 +29,7 @@
   <parent>
     <groupId>org.apache.slider</groupId>
     <artifactId>slider</artifactId>
-    <version>0.40</version>
+    <version>0.41.0-SNAPSHOT</version>
     <relativePath>../../../</relativePath>
   </parent>
 


[15/50] [abbrv] git commit: SLIDER-134. Provide a default ZK node for apps

Posted by sm...@apache.org.
SLIDER-134. Provide a default ZK node for apps


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

Branch: refs/heads/master
Commit: 8f933709d42034a541609874aaf3423f4b8904a8
Parents: a613e20
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Sun Jul 6 22:25:15 2014 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Mon Jul 7 07:49:32 2014 -0700

----------------------------------------------------------------------
 app-packages/hbase/appConfig.json               |   1 +
 .../org/apache/slider/client/SliderClient.java  | 112 ++++++++++++++++++-
 .../slider/core/zk/BlockingZKWatcher.java       |   8 +-
 .../apache/slider/core/zk/ZKIntegration.java    |  23 +++-
 .../slider/providers/agent/AgentKeys.java       |   1 +
 .../providers/agent/AgentProviderService.java   |   1 +
 .../common/tools/TestZKIntegration.groovy       |  53 +++++++++
 7 files changed, 191 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8f933709/app-packages/hbase/appConfig.json
----------------------------------------------------------------------
diff --git a/app-packages/hbase/appConfig.json b/app-packages/hbase/appConfig.json
index 9a43863..81fcf2c 100644
--- a/app-packages/hbase/appConfig.json
+++ b/app-packages/hbase/appConfig.json
@@ -5,6 +5,7 @@
   "global": {
     "agent.conf": "/slider/agent/conf/agent.ini",
     "application.def": "/slider/${app.package.name}.zip",
+    "create.default.zookeeper.node": "true",
     "config_types": "core-site,hdfs-site,hbase-site",
     "java_home": "/usr/jdk64/jdk1.7.0_45",
     "package_list": "files/hbase-${hbase.version}-bin.tar.gz",

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8f933709/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 c35a60a..286ebac 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
@@ -94,6 +94,8 @@ import org.apache.slider.core.registry.docstore.PublishedConfigurationOutputter;
 import org.apache.slider.core.registry.info.RegisteredEndpoint;
 import org.apache.slider.core.registry.info.ServiceInstanceData;
 import org.apache.slider.core.registry.retrieve.RegistryRetriever;
+import org.apache.slider.core.zk.BlockingZKWatcher;
+import org.apache.slider.core.zk.ZKIntegration;
 import org.apache.slider.core.zk.ZKPathBuilder;
 import org.apache.slider.providers.AbstractClientProvider;
 import org.apache.slider.providers.SliderProviderFactory;
@@ -106,6 +108,10 @@ import org.apache.slider.server.services.registry.SliderRegistryService;
 import org.apache.slider.server.services.utility.AbstractSliderLaunchedService;
 
 import static org.apache.slider.common.params.SliderActions.*;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -268,6 +274,84 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
     return exitCode;
   }
 
+  /**
+   * Delete the zookeeper node associated with the calling user and the cluster
+   **/
+  protected boolean deleteZookeeperNode(String clusterName) throws YarnException, IOException {
+    String user = getUsername();
+    String zkPath = ZKIntegration.mkClusterPath(user, clusterName);
+    try {
+      Configuration config = getConfig();
+      if (!SliderUtils.isHadoopClusterSecure(config)) {
+        ZKIntegration client = getZkClient(clusterName, user);
+        if (client != null) {
+          if (client.exists(zkPath)) {
+            log.info("Deleting zookeeper path {}", zkPath);
+          }
+          client.deleteRecursive(zkPath);
+          return true;
+        }
+      } else {
+        log.warn("Default zookeeper node is not available for secure cluster");
+      }
+    } catch (InterruptedException e) {
+      log.warn("Unable to recursively delete zk node {}", zkPath, e);
+    } catch (KeeperException e) {
+      log.warn("Unable to recursively delete zk node {}", zkPath, e);
+    } catch (BadConfigException e) {
+      log.warn("Unable to recursively delete zk node {}", zkPath, e);
+    }
+
+    return false;
+  }
+
+  /**
+   * Create the zookeeper node associated with the calling user and the cluster
+   */
+  protected String createZookeeperNode(String clusterName, Boolean nameOnly) throws YarnException, IOException {
+    String user = getUsername();
+    String zkPath = ZKIntegration.mkClusterPath(user, clusterName);
+    if(nameOnly) {
+      return zkPath;
+    }
+    Configuration config = getConfig();
+    if (!SliderUtils.isHadoopClusterSecure(config)) {
+      ZKIntegration client = getZkClient(clusterName, user);
+      if (client != null) {
+        try {
+          client.createPath(zkPath, "", ZooDefs.Ids.OPEN_ACL_UNSAFE,
+                            CreateMode.PERSISTENT);
+          return zkPath;
+        } catch (InterruptedException e) {
+          log.warn("Unable to create zk node {}", zkPath, e);
+        } catch (KeeperException e) {
+          log.warn("Unable to create zk node {}", zkPath, e);
+        }
+      }
+    }
+
+    return null;
+  }
+
+  /**
+   * Gets a zookeeper client, returns null if it cannot connect to zookeeper
+   **/
+  protected ZKIntegration getZkClient(String clusterName, String user) throws YarnException {
+    String registryQuorum = lookupZKQuorum();
+    ZKIntegration client = null;
+    try {
+      BlockingZKWatcher watcher = new BlockingZKWatcher();
+      client = ZKIntegration.newInstance(registryQuorum, user, clusterName, true, false, watcher);
+      client.init();
+      watcher.waitForZKConnection(2 * 1000);
+    } catch (InterruptedException e) {
+      client = null;
+      log.warn("Unable to connect to zookeeper quorum {}", registryQuorum, e);
+    } catch (IOException e) {
+      log.warn("Unable to connect to zookeeper quorum {}", registryQuorum, e);
+    }
+    return client;
+  }
 
   /**
    * Destroy a cluster. There's two race conditions here
@@ -297,6 +381,10 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
       log.warn("Filesystem returned false from delete() operation");
     }
 
+    if(!deleteZookeeperNode(clustername)) {
+      log.warn("Unable to perform node cleanup in Zookeeper.");
+    }
+
     List<ApplicationReport> instances = findAllLiveInstances(clustername);
     // detect any race leading to cluster creation during the check/destroy process
     // and report a problem.
@@ -382,7 +470,7 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    */
   
   public void buildInstanceDefinition(String clustername,
-                                         AbstractClusterBuildingActionArgs buildInfo)
+                                      AbstractClusterBuildingActionArgs buildInfo)
         throws YarnException, IOException {
     // verify that a live cluster isn't there
     SliderUtils.validateClusterName(clustername);
@@ -498,11 +586,25 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
         registryQuorum,
         quorum);
     String zookeeperRoot = buildInfo.getAppZKPath();
-    
+
     if (isSet(zookeeperRoot)) {
       zkPaths.setAppPath(zookeeperRoot);
-      
+    } else {
+      String createDefaultZkNode = appConf.getGlobalOptions().getOption(AgentKeys.CREATE_DEF_ZK_NODE, "false");
+      if (createDefaultZkNode.equals("true")) {
+        String defaultZKPath = createZookeeperNode(clustername, false);
+        log.info("ZK node created for application instance: {}.", defaultZKPath);
+        if (defaultZKPath != null) {
+          zkPaths.setAppPath(defaultZKPath);
+        }
+      } else {
+        // create AppPath if default is being used
+        String defaultZKPath = createZookeeperNode(clustername, true);
+        log.info("ZK node assigned to application instance: {}.", defaultZKPath);
+        zkPaths.setAppPath(defaultZKPath);
+      }
     }
+
     builder.addZKBinding(zkPaths);
 
     //then propagate any package URI
@@ -646,8 +748,8 @@ public class SliderClient extends AbstractSliderLaunchedService implements RunSe
    */
   public LaunchedApplication launchApplication(String clustername,
                                                Path clusterDirectory,
-                               AggregateConf instanceDefinition,
-                               boolean debugAM)
+                                               AggregateConf instanceDefinition,
+                                               boolean debugAM)
     throws YarnException, IOException {
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8f933709/slider-core/src/main/java/org/apache/slider/core/zk/BlockingZKWatcher.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/zk/BlockingZKWatcher.java b/slider-core/src/main/java/org/apache/slider/core/zk/BlockingZKWatcher.java
index 62ebff3..ca49888 100644
--- a/slider-core/src/main/java/org/apache/slider/core/zk/BlockingZKWatcher.java
+++ b/slider-core/src/main/java/org/apache/slider/core/zk/BlockingZKWatcher.java
@@ -23,6 +23,7 @@ import org.apache.zookeeper.Watcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.net.ConnectException;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 public class BlockingZKWatcher implements Watcher {
@@ -49,7 +50,8 @@ public class BlockingZKWatcher implements Watcher {
    * @param timeout timeout in millis
    */
 
-  public void waitForZKConnection(int timeout) throws InterruptedException {
+  public void waitForZKConnection(int timeout)
+      throws InterruptedException, ConnectException {
     synchronized (connectedFlag) {
       if (!connectedFlag.get()) {
         log.info("waiting for ZK event");
@@ -57,7 +59,9 @@ public class BlockingZKWatcher implements Watcher {
         connectedFlag.wait(timeout);
       }
     }
-    assert connectedFlag.get();
+    if (!connectedFlag.get()) {
+      throw new ConnectException("Unable to connect to ZK quorum");
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8f933709/slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java b/slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java
index 6270123..54aeb4f 100644
--- a/slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java
+++ b/slider-core/src/main/java/org/apache/slider/core/zk/ZKIntegration.java
@@ -250,13 +250,34 @@ public class ZKIntegration implements Watcher {
                                      KeeperException {
     try {
       zookeeper.delete(path, -1);
+      log.debug("Deleting {}", path);
       return true;
     } catch (KeeperException.NoNodeException ignored) {
       return false;
     }
   }
 
-/**
+  /**
+   * Recursively delete a node, does not throw exception if any node does not exist.
+   * @param path
+   * @return true if delete was successful
+   */
+  public boolean deleteRecursive(String path) throws KeeperException, InterruptedException {
+
+    try {
+      List<String> children = zookeeper.getChildren(path, false);
+      for (String child : children) {
+        deleteRecursive(path + "/" + child);
+      }
+      delete(path);
+    } catch (KeeperException.NoNodeException ignored) {
+      return false;
+    }
+
+    return true;
+  }
+
+  /**
  * Build the path to a cluster; exists once the cluster has come up.
  * Even before that, a ZK watcher could wait for it.
  * @param username user

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8f933709/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 333058d..019a8e6 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
@@ -88,6 +88,7 @@ public interface AgentKeys {
   String PACKAGE_LIST = "package_list";
   String WAIT_HEARTBEAT = "wait.heartbeat";
   String PYTHON_EXE = "python";
+  String CREATE_DEF_ZK_NODE = "create.default.zookeeper.node";
   String HEARTBEAT_MONITOR_INTERVAL = "heartbeat.monitor.interval";
   String AGENT_INSTANCE_DEBUG_DATA = "agent.instance.debug.data";
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8f933709/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 85aa8db..6cd3d9e 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
@@ -898,6 +898,7 @@ public class AgentProviderService extends AbstractProviderService implements
     tokens.put("${NN_URI}", nnuri);
     tokens.put("${NN_HOST}", URI.create(nnuri).getHost());
     tokens.put("${ZK_HOST}", appConf.get(OptionKeys.ZOOKEEPER_HOSTS));
+    tokens.put("${DEF_ZK_PATH}", appConf.get(OptionKeys.ZOOKEEPER_PATH));
     tokens.put("${DEFAULT_DATA_DIR}", getAmState()
         .getInternalsSnapshot()
         .getGlobalOptions()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/8f933709/slider-core/src/test/groovy/org/apache/slider/common/tools/TestZKIntegration.groovy
----------------------------------------------------------------------
diff --git a/slider-core/src/test/groovy/org/apache/slider/common/tools/TestZKIntegration.groovy b/slider-core/src/test/groovy/org/apache/slider/common/tools/TestZKIntegration.groovy
index 3930864..fe3bef7 100644
--- a/slider-core/src/test/groovy/org/apache/slider/common/tools/TestZKIntegration.groovy
+++ b/slider-core/src/test/groovy/org/apache/slider/common/tools/TestZKIntegration.groovy
@@ -20,6 +20,7 @@ package org.apache.slider.common.tools
 
 import groovy.util.logging.Slf4j
 import org.apache.hadoop.conf.Configuration
+import org.apache.slider.client.SliderClient
 import org.apache.slider.core.zk.ZKIntegration
 import org.apache.slider.test.KeysForTests
 import org.apache.slider.test.YarnZKMiniClusterTestBase
@@ -88,10 +89,62 @@ class TestZKIntegration extends YarnZKMiniClusterTestBase implements KeysForTest
            (c1.endsWith(clusters[1]) && c2.endsWith(clusters[0]))
   }
 
+  @Test
+  public void testCreateAndDeleteDefaultZKPath() throws Throwable {
+    MockSliderClient client = new MockSliderClient()
+
+    String path = client.createZookeeperNode("cl1", true)
+    ZKIntegration zki = client.getLastZKIntegration()
+
+    String zkPath = ZKIntegration.mkClusterPath(USERNAME, "cl1")
+    assert zkPath == "/services/slider/users/" + USERNAME + "/cl1", "zkPath must be as expected"
+    assert path == zkPath
+    assert zki == null, "ZKIntegration should be null."
+    zki = createZKIntegrationInstance(getZKBinding(), "cl1", true, false, 5000);
+    assert false == zki.exists(zkPath), "zkPath should not exist"
+
+    path = client.createZookeeperNode("cl1", false)
+    zki = client.getLastZKIntegration()
+    assert zkPath == "/services/slider/users/" + USERNAME + "/cl1", "zkPath must be as expected"
+    assert path == zkPath
+    assert true == zki.exists(zkPath), "zkPath must exist"
+    zki.createPath(zkPath, "/cn", ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT)
+    assert true == zki.exists(zkPath + "/cn"), "zkPath with child node must exist"
+    client.deleteZookeeperNode("cl1")
+    assert false == zki.exists(zkPath), "zkPath must not exist"
+
+  }
+
   public String createEphemeralChild(ZKIntegration zki, String userPath) {
     return zki.createPath(userPath, "/cluster-",
                           ZooDefs.Ids.OPEN_ACL_UNSAFE,
                           CreateMode.EPHEMERAL_SEQUENTIAL)
   }
 
+  class MockSliderClient extends SliderClient {
+    private ZKIntegration zki;
+
+    @Override
+    public String getUsername() {
+      return USERNAME
+    }
+
+    @Override
+    protected ZKIntegration getZkClient(String clusterName, String user) {
+      zki = createZKIntegrationInstance(getZKBinding(), "cl1", true, false, 5000)
+      return zki;
+    }
+
+    @Override
+    public synchronized Configuration getConfig() {
+      Configuration conf = new Configuration();
+      return conf;
+    }
+
+    public ZKIntegration getLastZKIntegration() {
+      return zki
+    }
+
+  }
+
 }


[39/50] [abbrv] git commit: SLIDER-131 enabling agent to AM one way and two way SSL

Posted by sm...@apache.org.
SLIDER-131 enabling agent to AM one way and two way SSL


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

Branch: refs/heads/master
Commit: 859ca521214c63c4d0956f98d001920705760bd0
Parents: 55aa2b1
Author: Jon Maron <jm...@hortonworks.com>
Authored: Wed Jul 9 15:36:59 2014 -0400
Committer: Jon Maron <jm...@hortonworks.com>
Committed: Wed Jul 9 15:36:59 2014 -0400

----------------------------------------------------------------------
 pom.xml                                         |  17 +-
 .../src/main/python/agent/AgentConfig.py        |   3 +
 .../src/main/python/agent/Controller.py         |  32 ++-
 slider-agent/src/main/python/agent/NetUtil.py   |   8 +-
 slider-agent/src/main/python/agent/main.py      |  15 +-
 slider-agent/src/main/python/agent/security.py  |   7 +-
 .../src/test/python/agent/TestController.py     |   4 +-
 slider-agent/src/test/python/agent/TestMain.py  |   8 +-
 .../src/test/python/agent/TestNetUtil.py        |   2 +-
 slider-core/pom.xml                             |   7 +
 .../java/org/apache/slider/api/StatusKeys.java  |   3 +
 .../org/apache/slider/common/SliderKeys.java    |  14 +
 .../apache/slider/common/tools/SliderUtils.java |   2 +-
 .../core/launch/ClasspathConstructor.java       |   1 -
 .../registry/info/CustomRegistryConstants.java  |   2 +-
 .../providers/AbstractProviderService.java      |   7 +-
 .../slider/providers/ProviderService.java       |   8 +-
 .../slider/providers/agent/AgentKeys.java       |   1 +
 .../providers/agent/AgentProviderService.java   |  16 +-
 .../slideram/SliderAMClientProvider.java        |   7 +-
 .../slideram/SliderAMProviderService.java       |  18 +-
 .../server/appmaster/SliderAppMaster.java       |  68 ++++-
 .../server/appmaster/web/AgentService.java      |  53 ++++
 .../server/appmaster/web/SliderAmIpFilter.java  |  18 +-
 .../slider/server/appmaster/web/WebAppApi.java  |   9 +-
 .../server/appmaster/web/WebAppApiImpl.java     |  16 +-
 .../appmaster/web/rest/AMWebServices.java       |   7 +-
 .../server/appmaster/web/rest/RestPaths.java    |   5 +-
 .../appmaster/web/rest/agent/AgentResource.java |  25 +-
 .../appmaster/web/rest/agent/AgentWebApp.java   | 203 +++++++++++++++
 .../web/rest/agent/AgentWebServices.java        |  40 +++
 .../services/security/CertificateManager.java   | 257 +++++++++++++++++++
 .../server/services/security/SecurityUtils.java | 209 +++++++++++++++
 .../services/security/SignCertResponse.java     |  67 +++++
 .../server/services/security/SignMessage.java   |  54 ++++
 .../providers/slideram/instance/appconf.json    |   5 +-
 .../main/resources/webapps/slideragent/.keep    |   0
 .../agent/TestAgentAMManagementWS.groovy        |  46 +++-
 .../model/mock/MockProviderService.groovy       |  12 +-
 .../view/TestClusterSpecificationBlock.groovy   |   2 +-
 .../web/view/TestContainerStatsBlock.groovy     |   2 +-
 .../appmaster/web/view/TestIndexBlock.groovy    |   2 +-
 .../agent/TestAgentProviderService.java         |   6 +-
 .../web/rest/agent/TestAMAgentWebServices.java  | 217 ++++++++--------
 .../management/TestAMManagementWebServices.java |   2 +-
 .../security/TestCertificateManager.java        |  60 +++++
 slider-core/src/test/python/agent/main.py       |   2 +
 .../providers/hbase/HBaseProviderService.java   |   5 +-
 48 files changed, 1381 insertions(+), 193 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 1ee78fe..a4489bb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -303,6 +303,7 @@
             <exclude>**/dfs.exclude</exclude>
             <exclude>**/*.iml</exclude>
             <exclude>**/rat.txt</exclude>
+            <exclude>**/get-hbase-site.sh</exclude>
             <exclude>DISCLAIMER</exclude>
             <exclude>app-packages/hbase/target/**</exclude>
           </excludes>
@@ -1140,9 +1141,9 @@
       </dependency>
 
 
-  <!-- ======================================================== -->
-  <!-- Mocking -->
-  <!-- ======================================================== -->
+      <!-- ======================================================== -->
+      <!-- Mocking -->
+      <!-- ======================================================== -->
 
       <dependency>
         <groupId>org.mockito</groupId>
@@ -1156,6 +1157,16 @@
         <version>${easymock.version}</version>
       </dependency>
 
+      <!-- ======================================================== -->
+      <!-- Jetty -->
+      <!-- ======================================================== -->
+
+      <dependency>
+	      <groupId>org.mortbay.jetty</groupId>
+	      <artifactId>jetty-sslengine</artifactId>
+        <version>6.1.26</version>
+      </dependency>
+
     </dependencies>
   </dependencyManagement>
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-agent/src/main/python/agent/AgentConfig.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/main/python/agent/AgentConfig.py b/slider-agent/src/main/python/agent/AgentConfig.py
index 91adfdd..16b924c 100644
--- a/slider-agent/src/main/python/agent/AgentConfig.py
+++ b/slider-agent/src/main/python/agent/AgentConfig.py
@@ -60,6 +60,9 @@ max_retries=2
 sleep_between_retries=1
 
 [security]
+keysdir=security/keys
+server_crt=ca.crt
+passphrase_env_var_name=SLIDER_PASSPHRASE
 
 [heartbeat]
 state_interval=6

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-agent/src/main/python/agent/Controller.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/main/python/agent/Controller.py b/slider-agent/src/main/python/agent/Controller.py
index b5dca92..92e9086 100644
--- a/slider-agent/src/main/python/agent/Controller.py
+++ b/slider-agent/src/main/python/agent/Controller.py
@@ -37,6 +37,7 @@ from NetUtil import NetUtil
 import ssl
 import ProcessHelper
 import Constants
+import security
 
 
 logger = logging.getLogger()
@@ -57,10 +58,10 @@ class Controller(threading.Thread):
     self.credential = None
     self.config = config
     self.hostname = config.getLabel()
-    server_url = 'http://' + config.get(AgentConfig.SERVER_SECTION,
+    server_url = 'https://' + config.get(AgentConfig.SERVER_SECTION,
                                         'hostname') + \
                  ':' + config.get(AgentConfig.SERVER_SECTION,
-                                  'port')
+                                  'secured_port')
     self.registerUrl = server_url + '/ws/v1/slider/agents/' + self.hostname + '/register'
     self.heartbeatUrl = server_url + '/ws/v1/slider/agents/' + self.hostname + '/heartbeat'
     self.netutil = NetUtil()
@@ -111,7 +112,8 @@ class Controller(threading.Thread):
     while not self.isRegistered:
       try:
         data = json.dumps(self.register.build(id))
-        logger.info("Registering with the server " + pprint.pformat(data))
+        logger.info("Registering with the server at " + self.registerUrl +
+                    " with data " + pprint.pformat(data))
         response = self.sendRequest(self.registerUrl, data)
         ret = json.loads(response)
         exitstatus = 0
@@ -395,11 +397,25 @@ class Controller(threading.Thread):
     pass
 
   def sendRequest(self, url, data):
-    req = urllib2.Request(url, data, {'Content-Type': 'application/json'})
-    f = urllib2.urlopen(req)
-    response = f.read()
-    f.close()
-    return response
+    response = None
+    try:
+        if self.cachedconnect is None: # Lazy initialization
+            self.cachedconnect = security.CachedHTTPSConnection(self.config)
+        req = urllib2.Request(url, data, {'Content-Type': 'application/json'})
+        response = self.cachedconnect.request(req)
+        return response
+    except Exception:
+        exc_type, exc_value, exc_traceback = sys.exc_info()
+        logger.error("Exception raised", exc_info=(exc_type, exc_value, exc_traceback))
+        if response is None:
+            err_msg = 'Request failed! Data: ' + str(data)
+            logger.warn(err_msg)
+            return {'exitstatus': 1, 'log': err_msg}
+        else:
+            err_msg = ('Response parsing failed! Request data: ' + str(data)
+                       + '; Response: ' + str(response))
+            logger.warn(err_msg)
+            return {'exitstatus': 1, 'log': err_msg}
 
 
 def main(argv=None):

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-agent/src/main/python/agent/NetUtil.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/main/python/agent/NetUtil.py b/slider-agent/src/main/python/agent/NetUtil.py
index ed8e687..eb658f7 100644
--- a/slider-agent/src/main/python/agent/NetUtil.py
+++ b/slider-agent/src/main/python/agent/NetUtil.py
@@ -18,6 +18,7 @@ from urlparse import urlparse
 import time
 import logging
 import httplib
+from ssl import SSLError
 
 logger = logging.getLogger()
 
@@ -37,7 +38,7 @@ class NetUtil:
     logger.info("Connecting to the following url " + url);
     try:
       parsedurl = urlparse(url)
-      ca_connection = httplib.HTTPConnection(parsedurl[1])
+      ca_connection = httplib.HTTPSConnection(parsedurl[1])
       ca_connection.request("GET", parsedurl[2])
       response = ca_connection.getresponse()  
       status = response.status    
@@ -47,6 +48,11 @@ class NetUtil:
         return True
       else: 
         return False
+    except SSLError as slerror:
+        logger.error(str(slerror))
+        logger.error("SSLError: Failed to connect. Please check openssl library versions. \n" +
+                     "Refer to: https://bugzilla.redhat.com/show_bug.cgi?id=1022468 for more details.")
+        return False
     except Exception, e:
       logger.info("Failed to connect to " + str(url) + " due to " + str(e))
       return False

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-agent/src/main/python/agent/main.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/main/python/agent/main.py b/slider-agent/src/main/python/agent/main.py
index 3632157..12e07ba 100644
--- a/slider-agent/src/main/python/agent/main.py
+++ b/slider-agent/src/main/python/agent/main.py
@@ -27,7 +27,6 @@ import traceback
 import os
 import time
 import errno
-import ConfigParser
 import ProcessHelper
 from Controller import Controller
 from AgentConfig import AgentConfig
@@ -40,7 +39,7 @@ agentPid = os.getpid()
 configFileRelPath = "infra/conf/agent.ini"
 logFileName = "agent.log"
 
-SERVER_STATUS_URL="http://{0}:{1}{2}"
+SERVER_STATUS_URL="https://{0}:{1}{2}"
 
 
 def signal_handler(signum, frame):
@@ -176,6 +175,7 @@ def main():
   parser.add_option("-l", "--label", dest="label", help="label of the agent", default=None)
   parser.add_option("--host", dest="host", help="AppMaster host", default=None)
   parser.add_option("--port", dest="port", help="AppMaster port", default=None)
+  parser.add_option("--secured_port", dest="secured_port", help="AppMaster 2 Way port", default=None)
   parser.add_option("--debug", dest="debug", help="Agent debug hint", default="")
   (options, args) = parser.parse_args()
 
@@ -201,12 +201,23 @@ def main():
   if options.port:
       agentConfig.set(AgentConfig.SERVER_SECTION, "port", options.port)
 
+  if options.secured_port:
+      agentConfig.set(AgentConfig.SERVER_SECTION, "secured_port", options.secured_port)
+
   if options.debug:
     agentConfig.set(AgentConfig.AGENT_SECTION, AgentConfig.APP_DBG_CMD, options.debug)
 
+  # set the security directory to a subdirectory of the run dir
+  secDir = os.path.join(agentConfig.getResolvedPath(AgentConfig.RUN_DIR), "security")
+  logger.info("Security/Keys directory: " + secDir)
+  agentConfig.set(AgentConfig.SECURITY_SECTION, "keysdir", secDir)
+
   logFile = os.path.join(agentConfig.getResolvedPath(AgentConfig.LOG_DIR), logFileName)
+
   perform_prestart_checks(agentConfig)
   ensure_folder_layout(agentConfig)
+  # create security dir if necessary
+  ensure_path_exists(secDir)
 
   setup_logging(options.verbose, logFile)
   update_log_level(agentConfig, logFile)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-agent/src/main/python/agent/security.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/main/python/agent/security.py b/slider-agent/src/main/python/agent/security.py
index 4037733..76671dc 100644
--- a/slider-agent/src/main/python/agent/security.py
+++ b/slider-agent/src/main/python/agent/security.py
@@ -140,7 +140,7 @@ class CertificateManager():
     self.keysdir = self.config.get('security', 'keysdir')
     self.server_crt=self.config.get('security', 'server_crt')
     self.server_url = 'https://' + self.config.get('server', 'hostname') + ':' \
-       + self.config.get('server', 'url_port')
+       + self.config.get('server', 'port')
     
   def getAgentKeyName(self):
     keysdir = self.config.get('security', 'keysdir')
@@ -187,7 +187,7 @@ class CertificateManager():
       logger.info("Agent certificate exists, ok")
             
   def loadSrvrCrt(self):
-    get_ca_url = self.server_url + '/cert/ca/'
+    get_ca_url = self.server_url + '/ws/v1/slider/agents/cert/ca/'
     logger.info("Downloading server cert from " + get_ca_url)
     stream = urllib2.urlopen(get_ca_url)
     response = stream.read()
@@ -196,7 +196,8 @@ class CertificateManager():
     srvr_crt_f.write(response)
       
   def reqSignCrt(self):
-    sign_crt_req_url = self.server_url + '/certs/' + hostname.hostname()
+    sign_crt_req_url = self.server_url + '/ws/v1/slider/agents/certs/' + \
+                       hostname.hostname()
     agent_crt_req_f = open(self.getAgentCrtReqName())
     agent_crt_req_content = agent_crt_req_f.read()
     passphrase_env_var = self.config.get('security', 'passphrase_env_var_name')

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-agent/src/test/python/agent/TestController.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/test/python/agent/TestController.py b/slider-agent/src/test/python/agent/TestController.py
index 8c671da..939e63f 100644
--- a/slider-agent/src/test/python/agent/TestController.py
+++ b/slider-agent/src/test/python/agent/TestController.py
@@ -262,7 +262,9 @@ class TestController(unittest.TestCase):
     data = "data"
     requestMock.return_value = conMock
 
-    self.assertEqual("response", self.controller.sendRequest(url, data))
+    expected = {'exitstatus': 1, 'log': 'Request failed! Data: ' + data}
+
+    self.assertEqual(expected, self.controller.sendRequest(url, data))
     requestMock.called_once_with(url, data,
       {'Content-Type': 'application/json'})
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-agent/src/test/python/agent/TestMain.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/test/python/agent/TestMain.py b/slider-agent/src/test/python/agent/TestMain.py
index 179d1b4..9ef1cad 100644
--- a/slider-agent/src/test/python/agent/TestMain.py
+++ b/slider-agent/src/test/python/agent/TestMain.py
@@ -259,10 +259,11 @@ class TestMain(unittest.TestCase):
     self.assertTrue(start_mock.called)
 
   class AgentOptions:
-      def __init__(self, label, host, port, verbose, debug):
+      def __init__(self, label, host, port, secured_port, verbose, debug):
           self.label = label
           self.host = host
           self.port = port
+          self.secured_port = secured_port
           self.verbose = verbose
           self.debug = debug
 
@@ -290,16 +291,17 @@ class TestMain(unittest.TestCase):
       Controller_init_mock.return_value = None
       isAlive_mock.return_value = False
       parse_args_mock.return_value = (
-          TestMain.AgentOptions("agent", "host1", "8080", True, ""), [])
+          TestMain.AgentOptions("agent", "host1", "8080", "8081", True, ""), [])
       tmpdir = tempfile.gettempdir()
 
       #testing call without command-line arguments
       os.environ["AGENT_WORK_ROOT"] = os.path.join(tmpdir, "work")
       os.environ["AGENT_LOG_ROOT"] = os.path.join(tmpdir, "log")
       main.main()
-      self.assertTrue(AgentConfig_set_mock.call_count == 2)
+      self.assertTrue(AgentConfig_set_mock.call_count == 4)
       AgentConfig_set_mock.assert_any_call("server", "hostname", "host1")
       AgentConfig_set_mock.assert_any_call("server", "port", "8080")
+      AgentConfig_set_mock.assert_any_call("server", "secured_port", "8081")
 
 
 if __name__ == "__main__":

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-agent/src/test/python/agent/TestNetUtil.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/test/python/agent/TestNetUtil.py b/slider-agent/src/test/python/agent/TestNetUtil.py
index c19ec19..550e148 100644
--- a/slider-agent/src/test/python/agent/TestNetUtil.py
+++ b/slider-agent/src/test/python/agent/TestNetUtil.py
@@ -25,7 +25,7 @@ import unittest
 class TestNetUtil(unittest.TestCase):
 
   @patch("urlparse.urlparse")
-  @patch("httplib.HTTPConnection")
+  @patch("httplib.HTTPSConnection")
   def test_checkURL(self, httpsConMock, parseMock):
 
     NetUtil.logger = MagicMock()

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/pom.xml
----------------------------------------------------------------------
diff --git a/slider-core/pom.xml b/slider-core/pom.xml
index ca09541..b6120fb 100644
--- a/slider-core/pom.xml
+++ b/slider-core/pom.xml
@@ -215,6 +215,7 @@
             <exclude>src/test/python/agent.ini</exclude>
             <exclude>src/test/python/version</exclude>
             <exclude>src/main/resources/webapps/slideram/.keep</exclude>
+            <exclude>src/main/resources/webapps/slideragent/.keep</exclude>
             <exclude>src/main/resources/webapps/static/yarn.dt.plugins.js</exclude>
             <!-- jQuery DataTables files (BSD license) -->
             <exclude>src/main/resources/webapps/static/dt-1.9.4/**</exclude>
@@ -507,6 +508,12 @@
       <scope>test</scope>
     </dependency>
 
+    <dependency>
+      <groupId>org.mortbay.jetty</groupId>
+      <artifactId>jetty-sslengine</artifactId>
+      <scope>compile</scope>
+    </dependency>
+
   </dependencies>
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java b/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
index 709c137..4bfcf41 100644
--- a/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
+++ b/slider-core/src/main/java/org/apache/slider/api/StatusKeys.java
@@ -67,4 +67,7 @@ public interface StatusKeys {
   String INFO_AM_RPC_PORT = "info.am.rpc.port";
   String INFO_AM_WEB_PORT = "info.am.web.port";
   String INFO_AM_WEB_URL = "info.am.web.url";
+  String INFO_AM_AGENT_PORT = "info.am.agent.port";
+  String INFO_AM_AGENT_URL = "info.am.agent.url";
+  String INFO_AM_SECURED_AGENT_PORT = "info.am.agent.secured.port";
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/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 a0e29c9..0ba562a 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
@@ -151,6 +151,20 @@ public interface SliderKeys extends SliderXmlConfKeys {
 
   String HADOOP_USER_NAME = "HADOOP_USER_NAME";
   String HADOOP_PROXY_USER = "HADOOP_PROXY_USER";
+  String SLIDER_PASSPHRASE = "SLIDER_PASSPHRASE";
 
   boolean PROPAGATE_RESOURCE_OPTION = true;
+
+  /**
+   * Security associated keys.
+   */
+  String SECURITY_DIR = "security";
+  String CRT_FILE_NAME = "ca.crt";
+  String CSR_FILE_NAME = "ca.csr";
+  String KEY_FILE_NAME = "ca.key";
+  String KEYSTORE_FILE_NAME = "keystore.p12";
+  String CRT_PASS_FILE_NAME = "pass.txt";
+  String PASSPHRASE = "DEV";
+  String PASS_LEN = "50";
+  String KEYSTORE_LOCATION = "ssl.server.keystore.location";
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
index 7c89321..17f8b70 100644
--- a/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
+++ b/slider-core/src/main/java/org/apache/slider/common/tools/SliderUtils.java
@@ -473,7 +473,7 @@ public final class SliderUtils {
     }
     return trailing? 
            b.toString()
-           : (b.substring(0, b.length() - 1));
+           : (b.substring(0, b.length() - separator.length()));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java b/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
index 5b74c03..3527149 100644
--- a/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
+++ b/slider-core/src/main/java/org/apache/slider/core/launch/ClasspathConstructor.java
@@ -24,7 +24,6 @@ import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.slider.common.tools.SliderUtils;
 
-import java.io.File;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java b/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
index 1eb87c6..38fb4a5 100644
--- a/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
+++ b/slider-core/src/main/java/org/apache/slider/core/registry/info/CustomRegistryConstants.java
@@ -32,7 +32,7 @@ public class CustomRegistryConstants {
       "org.apache.slider.publisher";
   
   public static final String AGENT_REST_API =
-      "org.apache.slider.publisher";
+      "org.apache.slider.agents";
 
   public static final String AM_IPC_PROTOCOL =
       "org.apache.slider.appmaster";

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java b/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
index a1940dd..e35227c 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/AbstractProviderService.java
@@ -312,11 +312,12 @@ public abstract class AbstractProviderService
     }
   }
   @Override
-  public void applyInitialRegistryDefinitions(URL amWebAPI,
-      ServiceInstanceData registryInstanceData) throws MalformedURLException,
+  public void applyInitialRegistryDefinitions(URL unsecureWebAPI,
+                                              URL secureWebAPI,
+                                              ServiceInstanceData registryInstanceData) throws MalformedURLException,
       IOException {
 
-      this.amWebAPI = amWebAPI;
+      this.amWebAPI = unsecureWebAPI;
     this.registryInstanceData = registryInstanceData;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java b/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
index 1778197..56e24e9 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/ProviderService.java
@@ -159,10 +159,12 @@ public interface ProviderService extends ProviderCore, Service,
 
   /**
    * Prior to going live -register the initial service registry data
-   * @param amWebAPI
+   * @param unsecureWebAPI
+   * @param secureWebAPI
    * @param registryInstanceData
    */
-  void applyInitialRegistryDefinitions(URL amWebAPI,
-      ServiceInstanceData registryInstanceData) throws MalformedURLException,
+  void applyInitialRegistryDefinitions(URL unsecureWebAPI,
+                                       URL secureWebAPI,
+                                       ServiceInstanceData registryInstanceData) throws MalformedURLException,
       IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/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 96acd41..31d09c4 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
@@ -67,6 +67,7 @@ public interface AgentKeys {
   String ARG_LABEL = "--label";
   String ARG_HOST = "--host";
   String ARG_PORT = "--port";
+  String ARG_SECURED_PORT = "--secured_port";
   String ARG_DEBUG = "--debug";
   String AGENT_MAIN_SCRIPT_ROOT = "./infra/agent/slider-agent/";
   String AGENT_MAIN_SCRIPT = "agent/main.py";

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/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 3430baf..c1719b7 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
@@ -209,6 +209,8 @@ public class AgentProviderService extends AbstractProviderService implements
     launcher.setEnv("AGENT_LOG_ROOT", logDir);
     log.info("AGENT_LOG_ROOT set to {}", logDir);
     launcher.setEnv(HADOOP_USER_NAME, System.getenv(HADOOP_USER_NAME));
+    // for 2-Way SSL
+    launcher.setEnv(SLIDER_PASSPHRASE, SliderKeys.PASSPHRASE);
 
     //local resources
 
@@ -261,7 +263,9 @@ public class AgentProviderService extends AbstractProviderService implements
     operation.add(ARG_HOST);
     operation.add(getClusterInfoPropertyValue(StatusKeys.INFO_AM_HOSTNAME));
     operation.add(ARG_PORT);
-    operation.add(getClusterInfoPropertyValue(StatusKeys.INFO_AM_WEB_PORT));
+    operation.add(getClusterInfoPropertyValue(StatusKeys.INFO_AM_AGENT_PORT));
+    operation.add(ARG_SECURED_PORT);
+    operation.add(getClusterInfoPropertyValue(StatusKeys.INFO_AM_SECURED_AGENT_PORT));
 
     String debugCmd = agentLaunchParameter.getNextLaunchParameter(role);
     if (debugCmd != null && debugCmd.length() != 0) {
@@ -986,15 +990,19 @@ public class AgentProviderService extends AbstractProviderService implements
   }
 
   @Override
-  public void applyInitialRegistryDefinitions(URL amWebAPI,
+  public void applyInitialRegistryDefinitions(URL unsecureWebAPI,
+                                              URL secureWebAPI,
                                               ServiceInstanceData instanceData) throws IOException {
-    super.applyInitialRegistryDefinitions(amWebAPI, instanceData);
+    super.applyInitialRegistryDefinitions(unsecureWebAPI,
+                                          secureWebAPI,
+                                          instanceData
+    );
 
     try {
       instanceData.internalView.endpoints.put(
           CustomRegistryConstants.AGENT_REST_API,
           new RegisteredEndpoint(
-              new URL(amWebAPI, SLIDER_PATH_AGENTS),
+              new URL(secureWebAPI, SLIDER_PATH_AGENTS),
               "Agent REST API"));
     } catch (URISyntaxException e) {
       throw new IOException(e);

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java b/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
index 6b40856..6aeb801 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMClientProvider.java
@@ -44,6 +44,7 @@ import org.apache.slider.providers.AbstractClientProvider;
 import org.apache.slider.providers.PlacementPolicy;
 import org.apache.slider.providers.ProviderRole;
 import org.apache.slider.providers.ProviderUtils;
+import org.mortbay.jetty.security.SslSelectChannelConnector;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -160,7 +161,8 @@ public class SliderAMClientProvider extends AbstractClientProvider implements
     Class<?>[] classes = {
       JCommander.class,
       GsonBuilder.class,
-      
+      SslSelectChannelConnector.class,
+
       CuratorFramework.class,
       CuratorZookeeperClient.class,
       ServiceInstance.class,
@@ -171,7 +173,8 @@ public class SliderAMClientProvider extends AbstractClientProvider implements
       {
         JCOMMANDER_JAR,
         GSON_JAR,
-        
+        "jetty-sslengine.jar",
+
         "curator-framework.jar",
         "curator-client.jar",
         "curator-x-discovery.jar",

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java b/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
index 1610954..184c25a 100644
--- a/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
+++ b/slider-core/src/main/java/org/apache/slider/providers/slideram/SliderAMProviderService.java
@@ -110,9 +110,13 @@ public class SliderAMProviderService extends AbstractProviderService implements
   }
 
   @Override
-  public void applyInitialRegistryDefinitions(URL amWebAPI,
-      ServiceInstanceData instanceData) throws IOException {
-    super.applyInitialRegistryDefinitions(amWebAPI, instanceData);
+  public void applyInitialRegistryDefinitions(URL unsecureWebAPI,
+                                              URL secureWebAPI,
+                                              ServiceInstanceData instanceData) throws IOException {
+    super.applyInitialRegistryDefinitions(unsecureWebAPI,
+                                          secureWebAPI,
+                                          instanceData
+    );
 
     // now publish site.xml files
     YarnConfiguration defaultYarnConfig = new YarnConfiguration();
@@ -146,24 +150,24 @@ public class SliderAMProviderService extends AbstractProviderService implements
     try {
       RegistryView externalView = instanceData.externalView;
       RegisteredEndpoint webUI =
-          new RegisteredEndpoint(amWebAPI, "Application Master Web UI");
+          new RegisteredEndpoint(unsecureWebAPI, "Application Master Web UI");
 
       externalView.endpoints.put(CommonRegistryConstants.WEB_UI, webUI);
 
       externalView.endpoints.put(
           CustomRegistryConstants.MANAGEMENT_REST_API,
           new RegisteredEndpoint(
-              new URL(amWebAPI, SLIDER_PATH_MANAGEMENT),
+              new URL(unsecureWebAPI, SLIDER_PATH_MANAGEMENT),
               "Management REST API") );
 
       externalView.endpoints.put(
           CustomRegistryConstants.REGISTRY_REST_API,
           new RegisteredEndpoint(
-              new URL(amWebAPI, RestPaths.SLIDER_PATH_REGISTRY + "/" +
+              new URL(unsecureWebAPI, RestPaths.SLIDER_PATH_REGISTRY + "/" +
                                 RestPaths.REGISTRY_SERVICE),
               "Registry Web Service" ) );
 
-      URL publisherURL = new URL(amWebAPI, SLIDER_PATH_PUBLISHER);
+      URL publisherURL = new URL(unsecureWebAPI, SLIDER_PATH_PUBLISHER);
       externalView.endpoints.put(
           CustomRegistryConstants.PUBLISHER_REST_API,
           new RegisteredEndpoint(

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
index 739d22c..0b22910 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/SliderAppMaster.java
@@ -105,6 +105,8 @@ import org.apache.slider.server.appmaster.state.ProviderAppState;
 import org.apache.slider.server.appmaster.state.RMOperationHandler;
 import org.apache.slider.server.appmaster.state.RoleInstance;
 import org.apache.slider.server.appmaster.state.RoleStatus;
+import org.apache.slider.server.appmaster.web.AgentService;
+import org.apache.slider.server.appmaster.web.rest.agent.AgentWebApp;
 import org.apache.slider.server.appmaster.web.SliderAMWebApp;
 import org.apache.slider.server.appmaster.web.SliderAmFilterInitializer;
 import org.apache.slider.server.appmaster.web.SliderAmIpFilter;
@@ -112,9 +114,10 @@ import org.apache.slider.server.appmaster.web.WebAppApi;
 import org.apache.slider.server.appmaster.web.WebAppApiImpl;
 import org.apache.slider.server.appmaster.web.rest.RestPaths;
 import org.apache.slider.server.services.registry.SliderRegistryService;
+import org.apache.slider.server.services.security.CertificateManager;
 import org.apache.slider.server.services.utility.AbstractSliderLaunchedService;
-import org.apache.slider.server.services.workflow.WorkflowRpcService;
 import org.apache.slider.server.services.utility.WebAppService;
+import org.apache.slider.server.services.workflow.WorkflowRpcService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -123,6 +126,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URL;
+import java.net.URLClassLoader;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -135,6 +139,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.ReentrantLock;
 
+import static org.apache.slider.server.appmaster.web.rest.RestPaths.WS_AGENT_CONTEXT_ROOT;
 import static org.apache.slider.server.appmaster.web.rest.RestPaths.WS_CONTEXT_ROOT;
 
 /**
@@ -307,6 +312,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
   private InetSocketAddress rpcServiceAddress;
   private ProviderService sliderAMProvider;
+  private String agentAccessUrl;
+  private CertificateManager certificateManager;
 
   /**
    * Service Constructor
@@ -473,7 +480,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     // Try to get the proper filtering of static resources through the yarn proxy working
     serviceConf.set(HADOOP_HTTP_FILTER_INITIALIZERS,
                     SliderAmFilterInitializer.NAME);
-    serviceConf.set(SliderAmIpFilter.WS_CONTEXT_ROOT, WS_CONTEXT_ROOT);
+    serviceConf.set(SliderAmIpFilter.WS_CONTEXT_ROOT, WS_CONTEXT_ROOT + "|" + WS_AGENT_CONTEXT_ROOT);
     
     //get our provider
     MapOperations globalInternalOptions =
@@ -590,10 +597,20 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       providerRoles.addAll(SliderAMClientProvider.ROLES);
 
       // Start up the WebApp and track the URL for it
+      certificateManager = new CertificateManager();
+      certificateManager.initRootCert(
+          instanceDefinition.getAppConfOperations()
+              .getComponent(SliderKeys.COMPONENT_AM));
+
+      startAgentWebApp(appInformation, serviceConf);
+
       webApp = new SliderAMWebApp(registry);
       WebApps.$for(SliderAMWebApp.BASE_PATH, WebAppApi.class,
-          new WebAppApiImpl(this, stateForProviders, providerService),
-          RestPaths.WS_CONTEXT)
+                   new WebAppApiImpl(this,
+                                     stateForProviders,
+                                     providerService,
+                                     certificateManager),
+                   RestPaths.WS_CONTEXT)
                       .with(serviceConf)
                       .start(webApp);
       appMasterTrackingUrl = "http://" + appMasterHostname + ":" + webApp.port();
@@ -605,7 +622,7 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
       addService(webAppService);
 
       appInformation.put(StatusKeys.INFO_AM_WEB_URL, appMasterTrackingUrl + "/");
-      appInformation.set(StatusKeys.INFO_AM_WEB_PORT, webApp.port());      
+      appInformation.set(StatusKeys.INFO_AM_WEB_PORT, webApp.port());
 
       // Register self with ResourceManager
       // This will start heartbeating to the RM
@@ -723,6 +740,32 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
     return amExitCode;
   }
 
+  private void startAgentWebApp(MapOperations appInformation,
+                                Configuration serviceConf) {
+    LOG_YARN.info("AM classpath:" + ((URLClassLoader) AgentWebApp.class.getClassLoader() ).getURLs());
+    // Start up the agent web app and track the URL for it
+    AgentWebApp agentWebApp = AgentWebApp.$for(AgentWebApp.BASE_PATH,
+                     new WebAppApiImpl(this,
+                                       stateForProviders,
+                                       providerService,
+                                       certificateManager),
+                     RestPaths.AGENT_WS_CONTEXT)
+        .withComponentConfig(getInstanceDefinition().getAppConfOperations()
+                                 .getComponent(SliderKeys.COMPONENT_AM))
+        .start();
+    agentAccessUrl = "https://" + appMasterHostname + ":" + agentWebApp.getSecuredPort();
+    AgentService agentService =
+      new AgentService("slider-agent", agentWebApp);
+
+    agentService.init(serviceConf);
+    agentService.start();
+    addService(agentService);
+
+    appInformation.put(StatusKeys.INFO_AM_AGENT_URL, agentAccessUrl + "/");
+    appInformation.set(StatusKeys.INFO_AM_AGENT_PORT, agentWebApp.getPort());
+    appInformation.set(StatusKeys.INFO_AM_SECURED_AGENT_PORT,
+                       agentWebApp.getSecuredPort());
+  }
 
   /**
    * This registers the service instance and its external values
@@ -733,7 +776,8 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
   private void registerServiceInstance(String instanceName,
       ApplicationId appid) throws Exception {
     // the registry is running, so register services
-    URL amWebAPI = new URL(appMasterTrackingUrl);
+    URL unsecureWebAPI = new URL(appMasterTrackingUrl);
+    URL secureWebAPI = new URL(agentAccessUrl);
     String serviceName = SliderKeys.APP_TYPE;
     int id = appid.getId();
     String appServiceType = RegistryNaming.createRegistryServiceType(
@@ -762,16 +806,22 @@ public class SliderAppMaster extends AbstractSliderLaunchedService
 
     // internal services
    
-    sliderAMProvider.applyInitialRegistryDefinitions(amWebAPI, instanceData);
+    sliderAMProvider.applyInitialRegistryDefinitions(unsecureWebAPI,
+                                                     secureWebAPI,
+                                                     instanceData
+    );
 
     // provider service dynamic definitions.
-    providerService.applyInitialRegistryDefinitions(amWebAPI, instanceData);
+    providerService.applyInitialRegistryDefinitions(unsecureWebAPI,
+                                                    secureWebAPI,
+                                                    instanceData
+    );
 
 
     // push the registration info to ZK
 
     registry.registerSelf(
-        instanceData, amWebAPI
+        instanceData, unsecureWebAPI
     );
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/server/appmaster/web/AgentService.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/AgentService.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/AgentService.java
new file mode 100644
index 0000000..08338e8
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/AgentService.java
@@ -0,0 +1,53 @@
+/*
+ * 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.server.appmaster.web;
+
+import org.apache.hadoop.service.AbstractService;
+import org.apache.slider.server.appmaster.web.rest.agent.AgentWebApp;
+
+/**
+ *
+ */
+public class AgentService extends AbstractService {
+  private volatile AgentWebApp webApp;
+
+  public AgentService(String name) {
+    super(name);
+  }
+
+  public AgentService(String name, AgentWebApp app) {
+    super(name);
+    webApp = app;
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+
+  }
+
+  /**
+   * Stop operation stops the webapp; sets the reference to null
+   * @throws Exception
+   */
+  @Override
+  protected void serviceStop() throws Exception {
+    if (webApp != null) {
+      webApp.stop();
+      webApp = null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAmIpFilter.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAmIpFilter.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAmIpFilter.java
index aba344e..ad5e219 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAmIpFilter.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/SliderAmIpFilter.java
@@ -36,7 +36,9 @@ import javax.servlet.http.HttpServletResponse;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
+import java.util.Arrays;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 
 public class SliderAmIpFilter implements Filter {
@@ -54,13 +56,13 @@ public class SliderAmIpFilter implements Filter {
   private Set<String> proxyAddresses = null;
   private long lastUpdate;
   private String proxyUriBase;
-  private String wsContextRoot;
+  private List<String> wsContextRoots;
   
   @Override
   public void init(FilterConfig conf) throws ServletException {
     proxyHost = conf.getInitParameter(PROXY_HOST);
     proxyUriBase = conf.getInitParameter(PROXY_URI_BASE);
-    wsContextRoot = conf.getInitParameter(WS_CONTEXT_ROOT);
+    wsContextRoots = Arrays.asList(conf.getInitParameter(WS_CONTEXT_ROOT).split("\\|"));
   }
   
   protected Set<String> getProxyAddresses() throws ServletException {
@@ -102,7 +104,7 @@ public class SliderAmIpFilter implements Filter {
       log.debug("Remote address for request is: " + httpReq.getRemoteAddr());
     }
     String requestURI = httpReq.getRequestURI();
-    if(!requestURI.startsWith(wsContextRoot) &&
+      if(!isWsRequest(requestURI) &&
        !getProxyAddresses().contains(httpReq.getRemoteAddr())) {
       String redirectUrl = httpResp.encodeRedirectURL(proxyUriBase +
                                                       requestURI);
@@ -136,4 +138,14 @@ public class SliderAmIpFilter implements Filter {
       throw e;
     }
   }
+
+  private boolean isWsRequest(String requestURI) {
+    boolean isWsReq = false;
+    for (String wsContext : wsContextRoots) {
+      isWsReq = requestURI.startsWith(wsContext);
+      if (isWsReq) break;
+    }
+
+    return isWsReq;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
index 4fac962..aa20baa 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApi.java
@@ -22,6 +22,7 @@ import org.apache.slider.server.appmaster.state.AppState;
 import org.apache.slider.server.appmaster.state.RoleStatus;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
 import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
+import org.apache.slider.server.services.security.CertificateManager;
 
 import java.util.Map;
 
@@ -39,7 +40,13 @@ public interface WebAppApi {
    * The {@link ProviderService} for the current cluster
    */
   public ProviderService getProviderService();
-  
+
+
+  /**
+   * The {@link CertificateManager} for the current cluster
+   */
+  public CertificateManager getCertificateManager();
+
   /**
    * The {@link SliderClusterProtocol} for the current cluster
    */

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
index 9a5a628..4eebd45 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/WebAppApiImpl.java
@@ -23,6 +23,7 @@ import org.apache.slider.providers.ProviderService;
 import org.apache.slider.server.appmaster.state.RoleStatus;
 import org.apache.slider.server.appmaster.state.StateAccessForProviders;
 import org.apache.slider.server.appmaster.web.rest.agent.AgentRestOperations;
+import org.apache.slider.server.services.security.CertificateManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,9 +46,12 @@ public class WebAppApiImpl implements WebAppApi {
   protected final SliderClusterProtocol clusterProto;
   protected final StateAccessForProviders appState;
   protected final ProviderService provider;
+  protected final CertificateManager certificateManager;
   
   public WebAppApiImpl(SliderClusterProtocol clusterProto,
-                       StateAccessForProviders appState, ProviderService provider) {
+                       StateAccessForProviders appState,
+                       ProviderService provider,
+                       CertificateManager certificateManager) {
     checkNotNull(clusterProto);
     checkNotNull(appState);
     checkNotNull(provider);
@@ -55,6 +59,7 @@ public class WebAppApiImpl implements WebAppApi {
     this.clusterProto = clusterProto;
     this.appState = appState;
     this.provider = provider;
+    this.certificateManager = certificateManager;
   }
 
   /* (non-Javadoc)
@@ -73,9 +78,14 @@ public class WebAppApiImpl implements WebAppApi {
     return provider;
   }
 
+  @Override
+  public CertificateManager getCertificateManager() {
+    return certificateManager;
+  }
+
   /* (non-Javadoc)
-   * @see org.apache.slider.server.appmaster.web.WebAppApi#getClusterProtocol()
-   */
+     * @see org.apache.slider.server.appmaster.web.WebAppApi#getClusterProtocol()
+     */
   @Override
   public SliderClusterProtocol getClusterProtocol() {
     return clusterProto;

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
index 91c83f2..4f068f3 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/AMWebServices.java
@@ -42,12 +42,7 @@ public class AMWebServices {
     return new ManagementResource(slider);
   }
 
-  @Path(RestPaths.SLIDER_SUBPATH_AGENTS)
-  public AgentResource getAgentResource () {
-    return new AgentResource(slider);
-  }
-
-  @Path(RestPaths.SLIDER_SUBPATH_PUBLISHER) 
+  @Path(RestPaths.SLIDER_SUBPATH_PUBLISHER)
   public PublisherResource getPublisherResource() {
     return new PublisherResource(slider);
   }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
index d55635f..0571ca1 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/RestPaths.java
@@ -24,15 +24,18 @@ package org.apache.slider.server.appmaster.web.rest;
 public class RestPaths {
 
   public static final String WS_CONTEXT = "ws";
+  public static final String AGENT_WS_CONTEXT = "ws";
   public static final String WS_CONTEXT_ROOT = "/" + WS_CONTEXT;
+  public static final String WS_AGENT_CONTEXT_ROOT = "/" + AGENT_WS_CONTEXT;
   public static final String SLIDER_CONTEXT_ROOT = WS_CONTEXT_ROOT +"/v1/slider";
+  public static final String SLIDER_AGENT_CONTEXT_ROOT = WS_AGENT_CONTEXT_ROOT +"/v1/slider";
   public static final String SLIDER_SUBPATH_MANAGEMENT = "/mgmt";
   public static final String SLIDER_SUBPATH_AGENTS = "/agents";
   public static final String SLIDER_SUBPATH_PUBLISHER = "/publisher";
 
   public static final String SLIDER_PATH_MANAGEMENT = SLIDER_CONTEXT_ROOT
                                       + SLIDER_SUBPATH_MANAGEMENT;
-  public static final String SLIDER_PATH_AGENTS = SLIDER_CONTEXT_ROOT
+  public static final String SLIDER_PATH_AGENTS = SLIDER_AGENT_CONTEXT_ROOT
                                       + SLIDER_SUBPATH_AGENTS;
   
   public static final String SLIDER_PATH_PUBLISHER = SLIDER_CONTEXT_ROOT

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
index 96b7b47..9d1e840 100644
--- a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentResource.java
@@ -17,9 +17,12 @@
 package org.apache.slider.server.appmaster.web.rest.agent;
 
 import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.services.security.SignCertResponse;
+import org.apache.slider.server.services.security.SignMessage;
 import org.codehaus.jackson.annotate.JsonIgnoreProperties;
 import org.codehaus.jackson.map.annotate.JsonSerialize;
 
+import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 import javax.ws.rs.Consumes;
 import javax.ws.rs.GET;
@@ -50,14 +53,14 @@ public class AgentResource {
   }
 
   @GET
-  @Path("/agents/register")
+  @Path("/agent/register")
   public Response endpointAgentRegister() {
     Response response = Response.status(200).entity("/agent/register").build();
     return response;
   }
 
   @GET
-  @Path("/agents")
+  @Path("/agent")
   public Response endpointAgent() {
     Response response = Response.status(200).entity("/agent").build();
     return response;
@@ -94,4 +97,22 @@ public class AgentResource {
     AgentRestOperations ops = slider.getAgentRestOperations();
     return ops.handleHeartBeat(message);
   }
+
+  @GET
+  @Path("/cert/ca")
+  @Produces({MediaType.TEXT_PLAIN})
+  public String downloadSrvrCrt() {
+    return slider.getCertificateManager().getServerCert();
+  }
+
+  @Path("/certs/{hostName}")
+  @POST
+  @Consumes(MediaType.APPLICATION_JSON)
+  @Produces({MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML})
+  public SignCertResponse signAgentCrt(@PathParam("hostName") String hostname,
+                                       SignMessage message, @Context HttpServletRequest req) {
+    return slider.getCertificateManager().signAgentCrt(hostname,
+                                                       message.getCsr(),
+                                                       message.getPassphrase());
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebApp.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebApp.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebApp.java
new file mode 100644
index 0000000..54d2b1f
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebApp.java
@@ -0,0 +1,203 @@
+/*
+ * 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.server.appmaster.web.rest.agent;
+
+import com.google.common.base.Preconditions;
+import com.sun.jersey.api.core.ResourceConfig;
+import com.sun.jersey.spi.container.WebApplication;
+import com.sun.jersey.spi.container.servlet.ServletContainer;
+import com.sun.jersey.spi.container.servlet.WebConfig;
+import com.sun.jersey.spi.inject.SingletonTypeInjectableProvider;
+import org.apache.slider.core.conf.MapOperations;
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.RestPaths;
+import org.apache.slider.server.services.security.SecurityUtils;
+import org.mortbay.jetty.Connector;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.security.SslSelectChannelConnector;
+import org.mortbay.jetty.servlet.Context;
+import org.mortbay.jetty.servlet.ServletHolder;
+import org.mortbay.thread.QueuedThreadPool;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.ws.rs.ext.Provider;
+import java.io.File;
+import java.util.Set;
+
+/**
+ *
+ */
+public class AgentWebApp {
+  protected static final Logger LOG = LoggerFactory.getLogger(AgentWebApp.class);
+  private int port;
+  private int securedPort;
+  private static Server agentServer;
+  public static final String BASE_PATH = "slideragent";
+
+  public static class Builder {
+    final String name;
+    final String wsName;
+    final WebAppApi application;
+    MapOperations configsMap;
+
+    public Builder(String name, String wsName, WebAppApi application) {
+      this.name = name;
+      this.wsName = wsName;
+      this.application = application;
+    }
+
+    public Builder withComponentConfig(MapOperations appMasterConfig) {
+      this.configsMap = appMasterConfig;
+      return this;
+    }
+
+    public AgentWebApp start() {
+      if (configsMap == null) {
+        throw new IllegalStateException("No SSL Configuration Available");
+      }
+
+      agentServer = new Server();
+      agentServer.setThreadPool(
+          new QueuedThreadPool(
+              configsMap.getOptionInt("agent.threadpool.size.max", 25)));
+      agentServer.setStopAtShutdown(true);
+
+      SslSelectChannelConnector ssl1WayConnector = createSSLConnector(false);
+      SslSelectChannelConnector ssl2WayConnector =
+          createSSLConnector(Boolean.valueOf(
+              configsMap.getOption("ssl.server.client.auth","false")));
+      agentServer.setConnectors(new Connector[]{ssl1WayConnector,
+          ssl2WayConnector});
+
+      ServletHolder agent = new ServletHolder(new AgentServletContainer());
+      Context agentRoot = new Context(agentServer, "/", Context.SESSIONS);
+
+      agent.setInitParameter("com.sun.jersey.config.property.resourceConfigClass",
+                             "com.sun.jersey.api.core.PackagesResourceConfig");
+      agent.setInitParameter("com.sun.jersey.config.property.packages",
+                             "org.apache.slider.server.appmaster.web.rest.agent");
+      agent.setInitParameter("com.sun.jersey.api.json.POJOMappingFeature",
+                             "true");
+//      agent.setInitParameter("com.sun.jersey.spi.container.ContainerRequestFilters", "com.sun.jersey.api.container.filter.LoggingFilter");
+//      agent.setInitParameter("com.sun.jersey.spi.container.ContainerResponseFilters", "com.sun.jersey.api.container.filter.LoggingFilter");
+//      agent.setInitParameter("com.sun.jersey.config.feature.Trace", "true");
+      agentRoot.addServlet(agent, "/*");
+
+      try {
+        agentServer.start();
+      } catch (Exception e) {
+        LOG.error("Unable to start agent server", e);
+      }
+
+      AgentWebApp webApp = new AgentWebApp();
+      webApp.setPort(getConnectorPort(agentServer, 0));
+      webApp.setSecuredPort(getConnectorPort(agentServer, 1));
+
+      return webApp;
+
+    }
+
+    private SslSelectChannelConnector createSSLConnector(boolean needClientAuth) {
+      SslSelectChannelConnector sslConnector = new
+          SslSelectChannelConnector();
+
+      String keystore = SecurityUtils.getSecurityDir() +
+                        File.separator + "keystore.p12";
+      String srvrCrtPass = SecurityUtils.getKeystorePass();
+      sslConnector.setKeystore(keystore);
+      sslConnector.setTruststore(keystore);
+      sslConnector.setPassword(srvrCrtPass);
+      sslConnector.setKeyPassword(srvrCrtPass);
+      sslConnector.setTrustPassword(srvrCrtPass);
+      sslConnector.setKeystoreType("PKCS12");
+      sslConnector.setTruststoreType("PKCS12");
+      sslConnector.setNeedClientAuth(needClientAuth);
+
+      sslConnector.setAcceptors(2);
+      return sslConnector;
+    }
+
+    @Provider
+    public class WebAppApiProvider extends
+        SingletonTypeInjectableProvider<javax.ws.rs.core.Context, WebAppApi> {
+
+      public WebAppApiProvider () {
+        super(WebAppApi.class, application);
+      }
+    }
+
+    public class AgentServletContainer extends ServletContainer {
+      public AgentServletContainer() {
+        super();
+      }
+
+      @Override
+      protected void configure(WebConfig wc,
+                               ResourceConfig rc,
+                               WebApplication wa) {
+        super.configure(wc, rc, wa);
+        Set<Object> singletons = rc.getSingletons();
+        singletons.add(new WebAppApiProvider());
+      }
+    }
+
+    private int getConnectorPort(Server webServer, int index) {
+      Preconditions.checkArgument(index >= 0);
+      if (index > webServer.getConnectors().length)
+        throw new IllegalStateException("Illegal connect index requested");
+
+      Connector c = webServer.getConnectors()[index];
+      if (c.getLocalPort() == -1) {
+        // The connector is not bounded
+        throw new IllegalStateException("The connector is not bound to a port");
+      }
+
+      return c.getLocalPort();
+    }
+  }
+
+  public static Builder $for(String name, WebAppApi app, String wsPrefix) {
+    return new Builder(name, wsPrefix, app);
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  public void setPort(int port) {
+    this.port = port;
+  }
+
+  public void setSecuredPort(int securedPort) {
+    this.securedPort = securedPort;
+  }
+
+  public int getSecuredPort() {
+    return securedPort;
+  }
+
+  public void stop() {
+    //need to stop server and reset injector
+    try {
+      agentServer.stop();
+    } catch (Exception e) {
+      LOG.warn("Unable to stop agent server", e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebServices.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebServices.java b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebServices.java
new file mode 100644
index 0000000..684ce6f
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/appmaster/web/rest/agent/AgentWebServices.java
@@ -0,0 +1,40 @@
+/*
+ * 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.server.appmaster.web.rest.agent;
+
+import org.apache.slider.server.appmaster.web.WebAppApi;
+import org.apache.slider.server.appmaster.web.rest.RestPaths;
+
+import javax.ws.rs.Path;
+import javax.ws.rs.core.Context;
+
+/** The available agent REST services exposed by a slider AM. */
+@Path(RestPaths.SLIDER_AGENT_CONTEXT_ROOT)
+public class AgentWebServices {
+  /** AM/WebApp info object */
+  @Context
+  private WebAppApi slider;
+
+  public AgentWebServices() {
+  }
+
+  @Path(RestPaths.SLIDER_SUBPATH_AGENTS)
+  public AgentResource getAgentResource () {
+    return new AgentResource(slider);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/server/services/security/CertificateManager.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/security/CertificateManager.java b/slider-core/src/main/java/org/apache/slider/server/services/security/CertificateManager.java
new file mode 100644
index 0000000..3771208
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/services/security/CertificateManager.java
@@ -0,0 +1,257 @@
+/**
+ * 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.server.services.security;
+
+import com.google.inject.Singleton;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.core.conf.MapOperations;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.Charset;
+import java.text.MessageFormat;
+
+@Singleton
+public class CertificateManager {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CertificateManager.class);
+
+  private static final String GEN_SRVR_KEY = "openssl genrsa -des3 " +
+      "-passout pass:{0} -out {1}/{2} 4096 ";
+  private static final String GEN_SRVR_REQ = "openssl req -passin pass:{0} " +
+      "-new -key {1}/{2} -out {1}/{5} -batch";
+  private static final String SIGN_SRVR_CRT = "openssl ca -create_serial " +
+    "-out {1}/{3} -days 365 -keyfile {1}/{2} -key {0} -selfsign " +
+    "-extensions jdk7_ca -config {1}/ca.config -batch " +
+    "-infiles {1}/{5}";
+  private static final String EXPRT_KSTR = "openssl pkcs12 -export" +
+      " -in {1}/{3} -inkey {1}/{2} -certfile {1}/{3} -out {1}/{4} " +
+      "-password pass:{0} -passin pass:{0} \n";
+  private static final String REVOKE_AGENT_CRT = "openssl ca " +
+      "-config {0}/ca.config -keyfile {0}/{4} -revoke {0}/{2} -batch " +
+      "-passin pass:{3} -cert {0}/{5}";
+  private static final String SIGN_AGENT_CRT = "openssl ca -config " +
+      "{0}/ca.config -in {0}/{1} -out {0}/{2} -batch -passin pass:{3} " +
+      "-keyfile {0}/{4} -cert {0}/{5}"; /**
+       * Verify that root certificate exists, generate it otherwise.
+       */
+  public void initRootCert(MapOperations compOperations) {
+    SecurityUtils.initializeSecurityParameters(compOperations);
+
+    LOG.info("Initialization of root certificate");
+    boolean certExists = isCertExists();
+    LOG.info("Certificate exists:" + certExists);
+
+    if (!certExists) {
+      generateServerCertificate();
+    }
+
+  }
+
+  /**
+   * Checks root certificate state.
+   * @return "true" if certificate exists
+   */
+  private boolean isCertExists() {
+
+    String srvrKstrDir = SecurityUtils.getSecurityDir();
+    String srvrCrtName = SliderKeys.CRT_FILE_NAME;
+    File certFile = new File(srvrKstrDir + File.separator + srvrCrtName);
+    LOG.debug("srvrKstrDir = " + srvrKstrDir);
+    LOG.debug("srvrCrtName = " + srvrCrtName);
+    LOG.debug("certFile = " + certFile.getAbsolutePath());
+
+    return certFile.exists();
+  }
+
+  /**
+   * Runs os command
+   *
+   * @return command execution exit code
+   */
+  private int runCommand(String command) {
+    String line = null;
+    Process process = null;
+    BufferedReader br= null;
+    try {
+      process = Runtime.getRuntime().exec(command);
+      br = new BufferedReader(new InputStreamReader(
+          process.getInputStream(), Charset.forName("UTF8")));
+
+      while ((line = br.readLine()) != null) {
+        LOG.info(line);
+      }
+
+      try {
+        process.waitFor();
+        SecurityUtils.logOpenSslExitCode(command, process.exitValue());
+        return process.exitValue(); //command is executed
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+    } catch (IOException e) {
+      e.printStackTrace();
+    } finally {
+      if (br != null) {
+        try {
+          br.close();
+        } catch (IOException ioe) {
+          ioe.printStackTrace();
+        }
+      }
+    }
+
+    return -1;//some exception occurred
+
+  }
+
+  private void generateServerCertificate() {
+    LOG.info("Generation of server certificate");
+
+    String srvrKstrDir = SecurityUtils.getSecurityDir();
+    String srvrCrtName = SliderKeys.CRT_FILE_NAME;
+    String srvrCsrName = SliderKeys.CSR_FILE_NAME;
+    String srvrKeyName = SliderKeys.KEY_FILE_NAME;
+    String kstrName = SliderKeys.KEYSTORE_FILE_NAME;
+    String srvrCrtPass = SecurityUtils.getKeystorePass();
+
+    Object[] scriptArgs = {srvrCrtPass, srvrKstrDir, srvrKeyName,
+        srvrCrtName, kstrName, srvrCsrName};
+
+    String command = MessageFormat.format(GEN_SRVR_KEY,scriptArgs);
+    runCommand(command);
+
+    command = MessageFormat.format(GEN_SRVR_REQ,scriptArgs);
+    runCommand(command);
+
+    command = MessageFormat.format(SIGN_SRVR_CRT,scriptArgs);
+    runCommand(command);
+
+    command = MessageFormat.format(EXPRT_KSTR,scriptArgs);
+    runCommand(command);
+
+  }
+
+  /**
+   * Returns server certificate content
+   * @return string with server certificate content
+   */
+  public String getServerCert() {
+    File certFile = new File(SecurityUtils.getSecurityDir() +
+        File.separator + SliderKeys.CRT_FILE_NAME);
+    String srvrCrtContent = null;
+    try {
+      srvrCrtContent = FileUtils.readFileToString(certFile);
+    } catch (IOException e) {
+      LOG.error(e.getMessage());
+    }
+    return srvrCrtContent;
+  }
+
+  /**
+   * Signs agent certificate
+   * Adds agent certificate to server keystore
+   * @return string with agent signed certificate content
+   */
+  public synchronized SignCertResponse signAgentCrt(String agentHostname,
+                                                    String agentCrtReqContent,
+                                                    String passphraseAgent) {
+    SignCertResponse response = new SignCertResponse();
+    LOG.info("Signing of agent certificate");
+    LOG.info("Verifying passphrase");
+
+    String passphraseSrvr = SliderKeys.PASSPHRASE;
+
+    if (!passphraseSrvr.equals(passphraseAgent.trim())) {
+      LOG.warn("Incorrect passphrase from the agent");
+      response.setResult(SignCertResponse.ERROR_STATUS);
+      response.setMessage("Incorrect passphrase from the agent");
+      return response;
+    }
+
+    String srvrKstrDir = SecurityUtils.getSecurityDir();
+    String srvrCrtPass = SecurityUtils.getKeystorePass();
+    String srvrCrtName = SliderKeys.CRT_FILE_NAME;
+    String srvrKeyName = SliderKeys.KEY_FILE_NAME;
+    String agentCrtReqName = agentHostname + ".csr";
+    String agentCrtName = agentHostname + ".crt";
+
+    Object[] scriptArgs = {srvrKstrDir, agentCrtReqName, agentCrtName,
+        srvrCrtPass, srvrKeyName, srvrCrtName};
+
+    //Revoke previous agent certificate if exists
+    File agentCrtFile = new File(srvrKstrDir + File.separator + agentCrtName);
+
+    if (agentCrtFile.exists()) {
+      LOG.info("Revoking of " + agentHostname + " certificate.");
+      String command = MessageFormat.format(REVOKE_AGENT_CRT, scriptArgs);
+      int commandExitCode = runCommand(command);
+      if (commandExitCode != 0) {
+        response.setResult(SignCertResponse.ERROR_STATUS);
+        response.setMessage(
+            SecurityUtils.getOpenSslCommandResult(command, commandExitCode));
+        return response;
+      }
+    }
+
+    File agentCrtReqFile = new File(srvrKstrDir + File.separator +
+        agentCrtReqName);
+    try {
+      FileUtils.writeStringToFile(agentCrtReqFile, agentCrtReqContent);
+    } catch (IOException e1) {
+      // TODO Auto-generated catch block
+      e1.printStackTrace();
+    }
+
+    String command = MessageFormat.format(SIGN_AGENT_CRT, scriptArgs);
+
+    LOG.debug(SecurityUtils.hideOpenSslPassword(command));
+
+    int commandExitCode = runCommand(command); // ssl command execution
+    if (commandExitCode != 0) {
+      response.setResult(SignCertResponse.ERROR_STATUS);
+      response.setMessage(
+          SecurityUtils.getOpenSslCommandResult(command, commandExitCode));
+      //LOG.warn(ShellCommandUtil.getOpenSslCommandResult(command, commandExitCode));
+      return response;
+    }
+
+    String agentCrtContent = "";
+    try {
+      agentCrtContent = FileUtils.readFileToString(agentCrtFile);
+    } catch (IOException e) {
+      e.printStackTrace();
+      LOG.error("Error reading signed agent certificate");
+      response.setResult(SignCertResponse.ERROR_STATUS);
+      response.setMessage("Error reading signed agent certificate");
+      return response;
+    }
+    response.setResult(SignCertResponse.OK_STATUS);
+    response.setSignedCa(agentCrtContent);
+    //LOG.info(ShellCommandUtil.getOpenSslCommandResult(command, commandExitCode));
+    return response;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/server/services/security/SecurityUtils.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/security/SecurityUtils.java b/slider-core/src/main/java/org/apache/slider/server/services/security/SecurityUtils.java
new file mode 100644
index 0000000..5238d90
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/services/security/SecurityUtils.java
@@ -0,0 +1,209 @@
+/*
+ * 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.server.services.security;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang.RandomStringUtils;
+import org.apache.slider.common.SliderKeys;
+import org.apache.slider.core.conf.MapOperations;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.attribute.PosixFilePermission;
+import java.nio.file.attribute.PosixFilePermissions;
+import java.util.Set;
+
+/**
+ *
+ */
+public class SecurityUtils {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SecurityUtils.class);
+
+  private static String CA_CONFIG_CONTENTS ="[ ca ]\n"
+                                            + "default_ca             = CA_CLIENT\n"
+                                            + "[ CA_CLIENT ]\n"
+                                            + "dir\t\t       = ${SEC_DIR}/db\n"
+                                            + "certs                  = $dir/certs\n"
+                                            + "new_certs_dir          = $dir/newcerts\n"
+                                            + "\n"
+                                            + "database               = $dir/index.txt\n"
+                                            + "serial                 = $dir/serial\n"
+                                            + "default_days           = 365    \n"
+                                            + "\n"
+                                            + "default_crl_days       = 7  \n"
+                                            + "default_md             = md5 \n"
+                                            + "\n"
+                                            + "policy                 = policy_anything \n"
+                                            + "\n"
+                                            + "[ policy_anything ]\n"
+                                            + "countryName            = optional\n"
+                                            + "stateOrProvinceName    = optional \n"
+                                            + "localityName           = optional\n"
+                                            + "organizationName       = optional\n"
+                                            + "organizationalUnitName = optional\n"
+                                            + "commonName             = optional   \n"
+                                            + "emailAddress           = optional       \n"
+                                            + "\n"
+                                            + "[ jdk7_ca ]\n"
+                                            + "subjectKeyIdentifier = hash\n"
+                                            + "authorityKeyIdentifier = keyid:always,issuer:always\n"
+                                            + "basicConstraints = CA:true";
+
+  private static final String PASS_TOKEN = "pass:";
+  private static String keystorePass;
+  private static String securityDir;
+
+  public static void logOpenSslExitCode(String command, int exitCode) {
+    if (exitCode == 0) {
+      LOG.info(getOpenSslCommandResult(command, exitCode));
+    } else {
+      LOG.warn(getOpenSslCommandResult(command, exitCode));
+    }
+
+  }
+
+  public static String hideOpenSslPassword(String command){
+    int start = command.indexOf(PASS_TOKEN)+PASS_TOKEN.length();
+    CharSequence cs = command.subSequence(start, command.indexOf(" ", start));
+    return command.replace(cs, "****");
+  }
+
+  public static String getOpenSslCommandResult(String command, int exitCode) {
+    return new StringBuilder().append("Command ").append(hideOpenSslPassword(command)).append(" was finished with exit code: ")
+        .append(exitCode).append(" - ").append(getOpenSslExitCodeDescription(exitCode)).toString();
+  }
+
+  private static String getOpenSslExitCodeDescription(int exitCode) {
+    switch (exitCode) {
+      case 0: {
+        return "the operation was completed successfully.";
+      }
+      case 1: {
+        return "an error occurred parsing the command options.";
+      }
+      case 2: {
+        return "one of the input files could not be read.";
+      }
+      case 3: {
+        return "an error occurred creating the PKCS#7 file or when reading the MIME message.";
+      }
+      case 4: {
+        return "an error occurred decrypting or verifying the message.";
+      }
+      case 5: {
+        return "the message was verified correctly but an error occurred writing out the signers certificates.";
+      }
+      default:
+        return "unsupported code";
+    }
+  }
+
+  public static void writeCaConfigFile(String path) throws IOException {
+    String contents = CA_CONFIG_CONTENTS.replace("${SEC_DIR}", path);
+    FileUtils.writeStringToFile(new File(path, "ca.config"), contents);
+  }
+
+  public static String getKeystorePass() {
+    return keystorePass;
+  }
+
+  public static String getSecurityDir() {
+    return securityDir;
+  }
+
+  public static void initializeSecurityParameters(MapOperations configMap) {
+    String keyStoreLocation = configMap.getOption(
+        SliderKeys.KEYSTORE_LOCATION, getDefaultKeystoreLocation());
+    File secDirFile = new File(keyStoreLocation).getParentFile();
+    if (!secDirFile.exists()) {
+      // create entire required directory structure
+      File dbDir = new File(secDirFile, "db");
+      File newCertsDir = new File(dbDir, "newcerts");
+      newCertsDir.mkdirs();
+      try {
+        Set<PosixFilePermission> perms =
+            PosixFilePermissions.fromString("rwx------");
+        Files.setPosixFilePermissions(Paths.get(secDirFile.toURI()), perms);
+        Files.setPosixFilePermissions(Paths.get(dbDir.toURI()), perms);
+        Files.setPosixFilePermissions(Paths.get(newCertsDir.toURI()), perms);
+        File indexFile = new File(dbDir, "index.txt");
+        indexFile.createNewFile();
+
+        SecurityUtils.writeCaConfigFile(secDirFile.getAbsolutePath());
+
+      } catch (IOException e) {
+        LOG.error("Unable to create SSL configuration directories/files", e);
+      }
+      // need to create the password
+    }
+    keystorePass = getKeystorePassword(secDirFile);
+    securityDir = secDirFile.getAbsolutePath();
+  }
+
+  private static String getKeystorePassword(File secDirFile) {
+    File passFile = new File(secDirFile, SliderKeys.CRT_PASS_FILE_NAME);
+    String password = null;
+
+    if (!passFile.exists()) {
+      LOG.info("Generation of file with password");
+      try {
+        password = RandomStringUtils.randomAlphanumeric(
+            Integer.valueOf(SliderKeys.PASS_LEN));
+        FileUtils.writeStringToFile(passFile, password);
+        passFile.setWritable(true);
+        passFile.setReadable(true);
+      } catch (IOException e) {
+        e.printStackTrace();
+        throw new RuntimeException(
+            "Error creating certificate password file");
+      }
+    } else {
+      LOG.info("Reading password from existing file");
+      try {
+        password = FileUtils.readFileToString(passFile);
+        password = password.replaceAll("\\p{Cntrl}", "");
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+    }
+
+    return password;
+  }
+
+  private static String getDefaultKeystoreLocation() {
+    Path workDir = null;
+    try {
+      workDir = Files.createTempDirectory("sec");
+    } catch (IOException e) {
+      LOG.warn("Unable to create security directory");
+      return null;
+    }
+
+    return new StringBuilder().append(workDir.toAbsolutePath())
+        .append(File.separator)
+        .append(SliderKeys.SECURITY_DIR)
+        .append(File.separator)
+        .append(SliderKeys.KEYSTORE_FILE_NAME).toString();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/859ca521/slider-core/src/main/java/org/apache/slider/server/services/security/SignCertResponse.java
----------------------------------------------------------------------
diff --git a/slider-core/src/main/java/org/apache/slider/server/services/security/SignCertResponse.java b/slider-core/src/main/java/org/apache/slider/server/services/security/SignCertResponse.java
new file mode 100644
index 0000000..8437d88
--- /dev/null
+++ b/slider-core/src/main/java/org/apache/slider/server/services/security/SignCertResponse.java
@@ -0,0 +1,67 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.slider.server.services.security;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlType;
+
+/**
+ *
+ * Sign certificate response data model.
+ *
+ */
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+@XmlType(name = "", propOrder = {})
+public class SignCertResponse {
+	
+  public static final String ERROR_STATUS = "ERROR";
+  public static final String OK_STATUS = "OK";
+
+  @XmlElement
+  private String result;
+  @XmlElement
+  private String signedCa;
+  @XmlElement
+  private String message;
+
+  public String getResult() {
+    return result;
+  }
+  public void setResult(String result) {
+    this.result = result;
+  }
+  public String getSignedCa() {
+    return signedCa;
+  }
+  public void setSignedCa(String signedCa) {
+    this.signedCa = signedCa;
+  }
+
+  public String getMessage() {
+    return message;
+  }
+  public void setMessage(String message) {
+    this.message = message;
+  }
+}
+