You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by ao...@apache.org on 2014/01/08 18:26:24 UTC

git commit: AMBARI-4241. Create yaml configurator and basic configurations for Storm (aonishuk)

Updated Branches:
  refs/heads/trunk ead9758ca -> 2999ec6c1


AMBARI-4241. Create yaml configurator and basic configurations for Storm
(aonishuk)


Project: http://git-wip-us.apache.org/repos/asf/ambari/repo
Commit: http://git-wip-us.apache.org/repos/asf/ambari/commit/2999ec6c
Tree: http://git-wip-us.apache.org/repos/asf/ambari/tree/2999ec6c
Diff: http://git-wip-us.apache.org/repos/asf/ambari/diff/2999ec6c

Branch: refs/heads/trunk
Commit: 2999ec6c192d57cdafd5a7e53432ecdf5f1dc199
Parents: ead9758
Author: Andrew Onischuk <ao...@hortonworks.com>
Authored: Wed Jan 8 09:24:44 2014 -0800
Committer: Andrew Onischuk <ao...@hortonworks.com>
Committed: Wed Jan 8 09:26:15 2014 -0800

----------------------------------------------------------------------
 .../services/STORM/configuration/storm-site.xml | 518 +++++++++++++++++++
 .../HDP/2.0.8/services/STORM/metainfo.xml       |  75 +++
 .../HDP/2.0.8/services/STORM/package/nimbus.py  |  68 +++
 .../HDP/2.0.8/services/STORM/package/params.py  |  25 +
 .../services/STORM/package/status_params.py     |  19 +
 .../2.0.8/services/STORM/package/supervisor.py  |  78 +++
 .../2.0.8/services/STORM/package/yaml_config.py |  19 +
 7 files changed, 802 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/2999ec6c/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/configuration/storm-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/configuration/storm-site.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/configuration/storm-site.xml
new file mode 100644
index 0000000..c6f0853
--- /dev/null
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/configuration/storm-site.xml
@@ -0,0 +1,518 @@
+<?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></description>
+  </property>
+  <property>
+    <name>storm.local.dir</name>
+    <value>"storm-local"</value>
+    <description>The place where jars are kept</description>
+  </property>
+  <property>
+    <name>storm.zookeeper.servers</name>
+    <value>"localhost"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.zookeeper.port</name>
+    <value>2181</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.zookeeper.root</name>
+    <value>"/storm"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.zookeeper.session.timeout</name>
+    <value>20000</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.zookeeper.connection.timeout</name>
+    <value>15000</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.zookeeper.retry.times</name>
+    <value>5</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.zookeeper.retry.interval</name>
+    <value>1000</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.zookeeper.retry.intervalceiling.millis</name>
+    <value>30000</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.cluster.mode</name>
+    <value>"distributed"</value>
+    <description>Can be distributed or local</description>
+  </property>
+  <property>
+    <name>storm.local.mode.zmq</name>
+    <value>false</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.thrift.transport</name>
+    <value>"backtype.storm.security.auth.SimpleTransportPlugin"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.messaging.transport</name>
+    <value>"backtype.storm.messaging.netty.Context"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.messaging.transport</name>
+    <value>"backtype.storm.messaging.netty.Context"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>nimbus.host</name>
+    <value>"localhost"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>nimbus.thrift.port</name>
+    <value>6627</value>
+    <description></description>
+  </property>
+  <property>
+    <name>nimbus.thrift.max_buffer_size</name>
+    <value>1048576</value>
+    <description></description>
+  </property>
+  <property>
+    <name>nimbus.childopts</name>
+    <value>"-Xmx1024m"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>nimbus.task.timeout.secs</name>
+    <value>30</value>
+    <description></description>
+  </property>
+  <property>
+    <name>nimbus.supervisor.timeout.secs</name>
+    <value>60</value>
+    <description></description>
+  </property>
+  <property>
+    <name>nimbus.monitor.freq.secs</name>
+    <value>10</value>
+    <description></description>
+  </property>
+  <property>
+    <name>nimbus.cleanup.inbox.freq.secs</name>
+    <value>600</value>
+    <description></description>
+  </property>
+  <property>
+    <name>nimbus.inbox.jar.expiration.secs</name>
+    <value>3600</value>
+    <description></description>
+  </property>
+  <property>
+    <name>nimbus.task.launch.secs</name>
+    <value>120</value>
+    <description></description>
+  </property>
+  <property>
+    <name>nimbus.reassign</name>
+    <value>true</value>
+    <description></description>
+  </property>
+  <property>
+    <name>nimbus.file.copy.expiration.secs</name>
+    <value>600</value>
+    <description></description>
+  </property>
+  <property>
+    <name>nimbus.topology.validator</name>
+    <value>"backtype.storm.nimbus.DefaultTopologyValidator"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>ui.port</name>
+    <value>8080</value>
+    <description></description>
+  </property>
+  <property>
+    <name>ui.childopts</name>
+    <value>"-Xmx768m"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>logviewer.port</name>
+    <value>8000</value>
+    <description></description>
+  </property>
+  <property>
+    <name>logviewer.port</name>
+    <value>8000</value>
+    <description></description>
+  </property>
+  <property>
+    <name>logviewer.childopts</name>
+    <value>"-Xmx128m"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>logviewer.appender.name</name>
+    <value>"A1"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>drpc.port</name>
+    <value>3772</value>
+    <description></description>
+  </property>
+  <property>
+    <name>drpc.worker.threads</name>
+    <value>64</value>
+    <description></description>
+  </property>
+  <property>
+    <name>drpc.queue.size</name>
+    <value>128</value>
+    <description></description>
+  </property>
+  <property>
+    <name>drpc.invocations.port</name>
+    <value>3773</value>
+    <description></description>
+  </property>
+  <property>
+    <name>drpc.request.timeout.secs</name>
+    <value>600</value>
+    <description></description>
+  </property>
+  <property>
+    <name>drpc.childopts</name>
+    <value>"-Xmx768m"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>transactional.zookeeper.root</name>
+    <value>"/transactional"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>transactional.zookeeper.servers</name>
+    <value>null</value>
+    <description></description>
+  </property>
+  <property>
+    <name>transactional.zookeeper.port</name>
+    <value>null</value>
+    <description></description>
+  </property>
+  <property>
+    <name>supervisor.slots.ports</name>
+    <value>
+    - 6700
+    - 6701
+    - 6702
+    - 6703</value>
+    <description></description>
+  </property>
+  <property>
+    <name>supervisor.childopts</name>
+    <value>"-Xmx256m"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>supervisor.worker.start.timeout.secs</name>
+    <value>120</value>
+    <description>How long supervisor will wait to ensure that a worker process is started</description>
+  </property>
+  <property>
+    <name>supervisor.worker.timeout.secs</name>
+    <value>30</value>
+    <description>How long between heartbeats until supervisor considers that worker dead and tries to restart it</description>
+  </property>
+  <property>
+    <name>supervisor.monitor.frequency.secs</name>
+    <value>3</value>
+    <description>How frequently the supervisor checks on the status of the processes it's monitoring and restarts if necessary</description>
+  </property>
+  <property>
+    <name>supervisor.heartbeat.frequency.secs</name>
+    <value>5</value>
+    <description>How frequently the supervisor heartbeats to the cluster state (for nimbus)</description>
+  </property>
+  <property>
+    <name>supervisor.enable</name>
+    <value>true</value>
+    <description></description>
+  </property>
+  <property>
+    <name>worker.childopts</name>
+    <value>"-Xmx768m"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>worker.heartbeat.frequency.secs</name>
+    <value>1</value>
+    <description></description>
+  </property>
+  <property>
+    <name>task.heartbeat.frequency.secs</name>
+    <value>3</value>
+    <description></description>
+  </property>
+  <property>
+    <name>task.refresh.poll.secs</name>
+    <value>10</value>
+    <description></description>
+  </property>
+  <property>
+    <name>zmq.threads</name>
+    <value>1</value>
+    <description></description>
+  </property>
+  <property>
+    <name>zmq.linger.millis</name>
+    <value>5000</value>
+    <description></description>
+  </property>
+  <property>
+    <name>zmq.hwm</name>
+    <value>0</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.messaging.netty.server_worker_threads</name>
+    <value>1</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.messaging.netty.client_worker_threads</name>
+    <value>1</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.messaging.netty.buffer_size</name>
+    <value>5242880</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.messaging.netty.max_retries</name>
+    <value>30</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.messaging.netty.max_wait_ms</name>
+    <value>1000</value>
+    <description></description>
+  </property>
+  <property>
+    <name>storm.messaging.netty.min_wait_ms</name>
+    <value>100</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.enable.message.timeouts</name>
+    <value>true</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.debug</name>
+    <value>false</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.optimize</name>
+    <value>true</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.workers</name>
+    <value>1</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.acker.executors</name>
+    <value>null</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.message.timeout.secs</name>
+    <value>30</value>
+    <description>Maximum amount of time a message has to complete before it's considered failed</description>
+  </property>
+  <property>
+    <name>topology.skip.missing.kryo.registrations</name>
+    <value>false</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.max.task.parallelism</name>
+    <value>null</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.max.spout.pending</name>
+    <value>null</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.state.synchronization.timeout.secs</name>
+    <value>60</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.stats.sample.rate</name>
+    <value>0.05</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.builtin.metrics.bucket.size.secs</name>
+    <value>60</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.fall.back.on.java.serialization</name>
+    <value>true</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.worker.childopts</name>
+    <value>null</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.executor.receive.buffer.size</name>
+    <value>1024</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.executor.send.buffer.size</name>
+    <value>1024</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.receiver.buffer.size</name>
+    <value>8</value>
+    <description>Setting it too high causes a lot of problems (heartbeat thread gets starved, throughput plummets)</description>
+  </property>
+  <property>
+    <name>topology.transfer.buffer.size</name>
+    <value>1024</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.tick.tuple.freq.secs</name>
+    <value>null</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.worker.shared.thread.pool.size</name>
+    <value>4</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.disruptor.wait.strategy</name>
+    <value>"com.lmax.disruptor.BlockingWaitStrategy"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.executor.send.buffer.size</name>
+    <value>1024</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.receiver.buffer.size</name>
+    <value>8</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.transfer.buffer.size</name>
+    <value>1024</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.tick.tuple.freq.secs</name>
+    <value>null</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.worker.shared.thread.pool.size</name>
+    <value>4</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.disruptor.wait.strategy</name>
+    <value>"com.lmax.disruptor.BlockingWaitStrategy"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.spout.wait.strategy</name>
+    <value>"backtype.storm.spout.SleepSpoutWaitStrategy"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.sleep.spout.wait.strategy.time.ms</name>
+    <value>1</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.error.throttle.interval.secs</name>
+    <value>10</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.max.error.report.per.interval</name>
+    <value>5</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.kryo.factory</name>
+    <value>"backtype.storm.serialization.DefaultKryoFactory"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.tuple.serializer</name>
+    <value>"backtype.storm.serialization.types.ListDelegateSerializer"</value>
+    <description></description>
+  </property>
+  <property>
+    <name>topology.trident.batch.emit.interval.millis</name>
+    <value>500</value>
+    <description></description>
+  </property>
+  <property>
+    <name>dev.zookeeper.path</name>
+    <value>"/tmp/dev-storm-zookeeper"</value>
+    <description></description>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/ambari/blob/2999ec6c/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/metainfo.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/metainfo.xml b/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/metainfo.xml
new file mode 100644
index 0000000..01a2d90
--- /dev/null
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/metainfo.xml
@@ -0,0 +1,75 @@
+<?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>
+  <services>
+    <service>
+      <name>STORM</name>
+      <comment>Apache Hadoop Stream processing framework</comment>
+      <version>0.9.0.1</version>
+      <components>
+
+        <component>
+          <name>NIMBUS</name>
+          <category>MASTER</category>
+          <commandScript>
+            <script>scripts/nimbus.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>
+      </components>
+      <!--
+      <osSpecifics>
+        <osSpecific>
+          <osType>any</osType>
+          <packages>
+            <package>
+              <type>rpm</type>
+              <name>storm</name>
+            </package>
+            <package>
+              <type>rpm</type>
+              <name>supervisor</name>
+            </package>
+        </osSpecific>
+      </osSpecifics> -->
+
+      <commandScript>
+        <script>scripts/service_check.py</script>
+        <scriptType>PYTHON</scriptType>
+        <timeout>300</timeout>
+      </commandScript>
+
+      <configuration-dependencies>
+        <config-type>storm-site</config-type>
+      </configuration-dependencies>
+    </service>
+  </services>
+</metainfo>

http://git-wip-us.apache.org/repos/asf/ambari/blob/2999ec6c/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/nimbus.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/nimbus.py b/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/nimbus.py
new file mode 100644
index 0000000..6ed1e39
--- /dev/null
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/nimbus.py
@@ -0,0 +1,68 @@
+#!/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 *
+
+         
+class Nimbus(Script):
+  def install(self, env):
+    self.install_packages(env)
+    self.configure(env)
+    
+  def configure(self, env):
+    import params
+    env.set_params(params)
+    
+    print "Configure."
+    
+  def start(self, env):
+    import params
+    env.set_params(params)
+    self.configure(env)
+
+    print "Start."
+    
+  def stop(self, env):
+    import params
+    env.set_params(params)
+
+    print "Stop."
+
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+    
+    #pid_file = format("{pid_dir}/?.pid")
+    #check_process_status(pid_file)
+
+# for testing
+def main():
+  command_type = "install"
+  command_data_file = '/root/storm.json'
+  basedir = '/root/ambari/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package'
+  stroutputf = '/1.txt'
+  sys.argv = ["", command_type, command_data_file, basedir, stroutputf]
+  
+  Nimbus().execute()
+  
+if __name__ == "__main__":
+  Nimbus().execute()
+  #main()

http://git-wip-us.apache.org/repos/asf/ambari/blob/2999ec6c/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/params.py b/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/params.py
new file mode 100644
index 0000000..b722187
--- /dev/null
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/params.py
@@ -0,0 +1,25 @@
+#!/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()

http://git-wip-us.apache.org/repos/asf/ambari/blob/2999ec6c/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/status_params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/status_params.py b/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/status_params.py
new file mode 100644
index 0000000..5561e10
--- /dev/null
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/status_params.py
@@ -0,0 +1,19 @@
+#!/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.
+
+"""

http://git-wip-us.apache.org/repos/asf/ambari/blob/2999ec6c/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/supervisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/supervisor.py b/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/supervisor.py
new file mode 100644
index 0000000..b4ce483
--- /dev/null
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/supervisor.py
@@ -0,0 +1,78 @@
+#!/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
+
+         
+class Supervisor(Script):
+  def install(self, env):
+    self.install_packages(env)
+    self.configure(env)
+    
+  def configure(self, env):
+    import params
+    env.set_params(params)
+    
+    # example
+    #yaml_config( "storm.yaml",
+    #        #conf_dir = params.conf_dir,
+    #        conf_dir = "/etc/storm/conf",
+    #        configurations = params.config['configurations']['storm-site'],
+    #        #owner = params.storm_user,
+    #        #group = params.user_group
+    #)
+    
+    print "Configure."
+    
+  def start(self, env):
+    import params
+    env.set_params(params)
+    self.configure(env)
+
+    print "Start."
+    
+  def stop(self, env):
+    import params
+    env.set_params(params)
+
+    print "Stop."
+
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+    
+    #pid_file = format("{pid_dir}/?.pid")
+    #check_process_status(pid_file)
+
+# for testing
+def main():
+  command_type = "install"
+  command_data_file = '/root/storm.json'
+  basedir = '/root/ambari/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package'
+  stroutputf = '/1.txt'
+  sys.argv = ["", command_type, command_data_file, basedir, stroutputf]
+  
+  Supervisor().execute()
+  
+if __name__ == "__main__":
+  Supervisor().execute()
+  #main()

http://git-wip-us.apache.org/repos/asf/ambari/blob/2999ec6c/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/yaml_config.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/yaml_config.py b/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/yaml_config.py
new file mode 100644
index 0000000..1e91ba1
--- /dev/null
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.8/services/STORM/package/yaml_config.py
@@ -0,0 +1,19 @@
+from resource_management import *
+
+def yaml_config(
+  filename,
+  configurations = None,
+  conf_dir = None,
+  mode = None,
+  owner = None,
+  group = None
+):
+    config_content = InlineTemplate('''{% for key, value in configurations_dict.items() %}{{ key }}: {{ value }}
+{% endfor %}''', configurations_dict=configurations)
+
+    File (format("{conf_dir}/{filename}"),
+      content = config_content,
+      owner = owner,
+      group = group,
+      mode = mode
+    )
\ No newline at end of file