You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by ab...@apache.org on 2015/04/10 13:07:53 UTC

ambari git commit: AMBARI-10365 - [WinTP2] Merge HDPWIN STORM package scripts to common services

Repository: ambari
Updated Branches:
  refs/heads/trunk 6cb623fd8 -> 4a9f83916


AMBARI-10365 - [WinTP2] Merge HDPWIN STORM package scripts to common services


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

Branch: refs/heads/trunk
Commit: 4a9f8391677c143cca8c11784438b22e7cb915cf
Parents: 6cb623f
Author: Artem Baranchuk <ab...@hortonworks.con>
Authored: Fri Apr 10 14:07:24 2015 +0300
Committer: Artem Baranchuk <ab...@hortonworks.con>
Committed: Fri Apr 10 14:07:24 2015 +0300

----------------------------------------------------------------------
 .../STORM/0.9.1.2.1/package/scripts/nimbus.py   |  50 +++--
 .../STORM/0.9.1.2.1/package/scripts/params.py   | 181 +----------------
 .../0.9.1.2.1/package/scripts/params_linux.py   | 198 +++++++++++++++++++
 .../0.9.1.2.1/package/scripts/params_windows.py |  29 +++
 .../0.9.1.2.1/package/scripts/service_check.py  |  17 ++
 .../0.9.1.2.1/package/scripts/status_params.py  |  50 ++---
 .../STORM/0.9.1.2.1/package/scripts/storm.py    |  14 +-
 .../0.9.1.2.1/package/scripts/supervisor.py     |  33 +++-
 .../0.9.1.2.1/package/scripts/ui_server.py      |  34 +++-
 .../0.9.1.2.1/package/scripts/yaml_utils.py     |  13 +-
 .../services/STORM/configuration/storm-env.xml  |   6 +-
 .../services/STORM/configuration/storm-site.xml |   7 -
 .../services/STORM/package/scripts/nimbus.py    |  50 -----
 .../services/STORM/package/scripts/params.py    |  30 ---
 .../STORM/package/scripts/service_check.py      |  32 ---
 .../STORM/package/scripts/service_mapping.py    |  22 ---
 .../STORM/package/scripts/status_params.py      |  22 ---
 .../2.1/services/STORM/package/scripts/storm.py |  33 ----
 .../STORM/package/scripts/supervisor.py         |  50 -----
 .../services/STORM/package/scripts/ui_server.py |  51 -----
 .../STORM/package/scripts/yaml_config.py        |  69 -------
 .../services/STORM/configuration/storm-site.xml |  12 +-
 22 files changed, 389 insertions(+), 614 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/nimbus.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/nimbus.py b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/nimbus.py
index 6c4fec3..98e9e9b 100644
--- a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/nimbus.py
+++ b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/nimbus.py
@@ -24,19 +24,17 @@ from resource_management.libraries.script import Script
 from resource_management.libraries.functions import format
 from resource_management.core.resources.system import Execute
 from resource_management.libraries.functions.version import compare_versions, format_hdp_stack_version
-
 from storm import storm
 from service import service
 from resource_management.libraries.functions.security_commons import build_expectations, \
   cached_kinit_executor, get_params_from_filesystem, validate_security_config_properties, \
   FILE_TYPE_JAAS_CONF
-from setup_ranger_storm import setup_ranger_storm  
-
-class Nimbus(Script):
+from setup_ranger_storm import setup_ranger_storm
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyImpl
 
-  def get_stack_to_component(self):
-    return {"HDP": "storm-nimbus"}
 
+class Nimbus(Script):
   def install(self, env):
     self.install_packages(env)
     self.configure(env)
@@ -44,13 +42,17 @@ class Nimbus(Script):
   def configure(self, env):
     import params
     env.set_params(params)
-
     storm()
 
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class NimbusDefault(Nimbus):
+  def get_stack_to_component(self):
+    return {"HDP": "storm-nimbus"}
+
   def pre_rolling_restart(self, env):
     import params
     env.set_params(params)
-
     if params.version and compare_versions(format_hdp_stack_version(params.version), '2.2.0.0') >= 0:
       Execute(format("hdp-select set storm-nimbus {version}"))
 
@@ -64,7 +66,6 @@ class Nimbus(Script):
   def stop(self, env, rolling_restart=False):
     import params
     env.set_params(params)
-
     service("nimbus", action="stop")
 
   def status(self, env):
@@ -74,26 +75,18 @@ class Nimbus(Script):
 
   def security_status(self, env):
     import status_params
-
     env.set_params(status_params)
-
     if status_params.security_enabled:
       # Expect the following files to be available in status_params.config_dir:
       #   storm_jaas.conf
-
       try:
         props_value_check = None
         props_empty_check = ['StormServer/keyTab', 'StormServer/principal']
         props_read_check = ['StormServer/keyTab']
-        storm_env_expectations = build_expectations('storm_jaas', props_value_check, props_empty_check,
-                                                 props_read_check)
-
+        storm_env_expectations = build_expectations('storm_jaas', props_value_check, props_empty_check,  props_read_check)
         storm_expectations = {}
         storm_expectations.update(storm_env_expectations)
-
-        security_params = get_params_from_filesystem(status_params.conf_dir,
-                                                     {'storm_jaas.conf': FILE_TYPE_JAAS_CONF})
-
+        security_params = get_params_from_filesystem(status_params.conf_dir, {'storm_jaas.conf': FILE_TYPE_JAAS_CONF})
         result_issues = validate_security_config_properties(security_params, storm_expectations)
         if not result_issues:  # If all validations passed successfully
           # Double check the dict before calling execute
@@ -124,5 +117,24 @@ class Nimbus(Script):
     else:
       self.put_structured_out({"securityState": "UNSECURED"})
 
+
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class NimbusWindows(Nimbus):
+  def start(self, env):
+    import status_params
+    env.set_params(status_params)
+    Service(status_params.nimbus_win_service_name, action="start")
+
+  def stop(self, env):
+    import status_params
+    env.set_params(status_params)
+    Service(status_params.nimbus_win_service_name, action="stop")
+
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+    check_windows_service_status(status_params.nimbus_win_service_name)
+
+
 if __name__ == "__main__":
   Nimbus().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params.py b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params.py
index d6e3881..a10c1d4 100644
--- a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params.py
@@ -17,182 +17,9 @@ See the License for the specific language governing permissions and
 limitations under the License.
 
 """
-from ambari_commons.constants import AMBARI_SUDO_BINARY
-from resource_management.libraries.functions.version import format_hdp_stack_version, compare_versions
-from resource_management.libraries.functions.default import default
-from resource_management.libraries.script import Script
-from resource_management.libraries.functions import default, format
-import status_params
-import re
+from ambari_commons import OSCheck
 
-def get_bare_principal(normalized_principal_name):
-  """
-  Given a normalized principal name (nimbus/c6501.ambari.apache.org@EXAMPLE.COM) returns just the
-  primary component (nimbus)
-  :param normalized_principal_name: a string containing the principal name to process
-  :return: a string containing the primary component value or None if not valid
-  """
-
-  bare_principal = None
-
-  if normalized_principal_name:
-    match = re.match(r"([^/@]+)(?:/[^@])?(?:@.*)?", normalized_principal_name)
-
-    if match:
-      bare_principal = match.group(1)
-
-  return bare_principal
-
-
-# server configurations
-config = Script.get_config()
-tmp_dir = Script.get_tmp_dir()
-sudo = AMBARI_SUDO_BINARY
-
-stack_name = default("/hostLevelParams/stack_name", None)
-
-version = default("/commandParams/version", None)
-
-stack_version_unformatted = str(config['hostLevelParams']['stack_version'])
-hdp_stack_version = format_hdp_stack_version(stack_version_unformatted)
-stack_is_hdp22_or_further = hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0
-
-#hadoop params
-if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
-  rest_lib_dir = '/usr/hdp/current/storm-client/contrib/storm-rest'
-  storm_bin_dir = "/usr/hdp/current/storm-client/bin"
-  storm_lib_dir = "/usr/hdp/current/storm-client/lib"
+if OSCheck.is_windows_family():
+  from params_windows import *
 else:
-  rest_lib_dir = "/usr/lib/storm/contrib/storm-rest"
-  storm_bin_dir = "/usr/bin"
-  storm_lib_dir = "/usr/lib/storm/lib/"
-
-storm_user = config['configurations']['storm-env']['storm_user']
-log_dir = config['configurations']['storm-env']['storm_log_dir']
-pid_dir = status_params.pid_dir
-conf_dir = "/etc/storm/conf"
-local_dir = config['configurations']['storm-site']['storm.local.dir']
-user_group = config['configurations']['cluster-env']['user_group']
-java64_home = config['hostLevelParams']['java_home']
-jps_binary = format("{java64_home}/bin/jps")
-nimbus_port = config['configurations']['storm-site']['nimbus.thrift.port']
-nimbus_host = config['configurations']['storm-site']['nimbus.host']
-rest_api_port = "8745"
-rest_api_admin_port = "8746"
-rest_api_conf_file = format("{conf_dir}/config.yaml")
-storm_env_sh_template = config['configurations']['storm-env']['content']
-jmxremote_port = config['configurations']['storm-env']['jmxremote_port']
-
-if 'ganglia_server_host' in config['clusterHostInfo'] and \
-    len(config['clusterHostInfo']['ganglia_server_host'])>0:
-  ganglia_installed = True
-  ganglia_server = config['clusterHostInfo']['ganglia_server_host'][0]
-  ganglia_report_interval = 60
-else:
-  ganglia_installed = False
-
-security_enabled = config['configurations']['cluster-env']['security_enabled']
-
-if security_enabled:
-  _hostname_lowercase = config['hostname'].lower()
-  _storm_principal_name = config['configurations']['storm-env']['storm_principal_name']
-  storm_jaas_principal = _storm_principal_name.replace('_HOST',_hostname_lowercase)
-  storm_keytab_path = config['configurations']['storm-env']['storm_keytab']
-
-  if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
-    storm_ui_keytab_path = config['configurations']['storm-env']['storm_ui_keytab']
-    _storm_ui_jaas_principal_name = config['configurations']['storm-env']['storm_ui_principal_name']
-    storm_ui_host = default("/clusterHostInfo/storm_ui_server_hosts", [])
-    storm_ui_jaas_principal = _storm_ui_jaas_principal_name.replace('_HOST',_hostname_lowercase)
-    
-    storm_bare_jaas_principal = get_bare_principal(_storm_principal_name)
-
-    _nimbus_principal_name = config['configurations']['storm-env']['nimbus_principal_name']
-    nimbus_jaas_principal = _nimbus_principal_name.replace('_HOST', _hostname_lowercase)
-    nimbus_bare_jaas_principal = get_bare_principal(_nimbus_principal_name)
-    nimbus_keytab_path = config['configurations']['storm-env']['nimbus_keytab']
-
-if stack_is_hdp22_or_further:
-  if security_enabled:
-    storm_thrift_transport = config['configurations']['storm-site']['_storm.thrift.secure.transport']
-  else:
-    storm_thrift_transport = config['configurations']['storm-site']['_storm.thrift.nonsecure.transport']
-
-ams_collector_hosts = default("/clusterHostInfo/metrics_collector_hosts", [])
-has_metric_collector = not len(ams_collector_hosts) == 0
-if has_metric_collector:
-  metric_collector_host = ams_collector_hosts[0]
-  metric_collector_report_interval = 60
-  metric_collector_app_id = "nimbus"
-metric_collector_sink_jar = "/usr/lib/storm/lib/ambari-metrics-storm-sink*.jar"
-
-# ranger host
-ranger_admin_hosts = default("/clusterHostInfo/ranger_admin_hosts", [])
-has_ranger_admin = not len(ranger_admin_hosts) == 0
-
-if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
-  # setting flag value for ranger hive plugin
-  enable_ranger_storm = False
-  ranger_plugin_enable = default("/configurations/ranger-storm-plugin-properties/ranger-storm-plugin-enabled", "no")
-  if ranger_plugin_enable.lower() == 'yes':
-    enable_ranger_storm = True
-  elif ranger_plugin_enable.lower() == 'no':
-    enable_ranger_storm = False
-
-ambari_server_hostname = config['clusterHostInfo']['ambari_server_host'][0]
-
-#ranger storm properties
-policymgr_mgr_url = default("/configurations/admin-properties/policymgr_external_url", "http://localhost:6080")
-sql_connector_jar = default("/configurations/admin-properties/SQL_CONNECTOR_JAR", "/usr/share/java/mysql-connector-java.jar")
-xa_audit_db_flavor = default("/configurations/admin-properties/DB_FLAVOR", "MYSQL")
-xa_audit_db_name = default("/configurations/admin-properties/audit_db_name", "ranger_audit")
-xa_audit_db_user = default("/configurations/admin-properties/audit_db_user", "rangerlogger")
-xa_audit_db_password = default("/configurations/admin-properties/audit_db_password", "rangerlogger")
-xa_db_host = default("/configurations/admin-properties/db_host", "localhost")
-repo_name = str(config['clusterName']) + '_storm'
-db_enabled = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.DB.IS_ENABLED", "false")
-hdfs_enabled = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.IS_ENABLED", "false")
-hdfs_dest_dir = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.DESTINATION_DIRECTORY", "hdfs://__REPLACE__NAME_NODE_HOST:8020/ranger/audit/app-type/time:yyyyMMdd")
-hdfs_buffer_dir = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.LOCAL_BUFFER_DIRECTORY", "__REPLACE__LOG_DIR/hadoop/app-type/audit")
-hdfs_archive_dir = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.LOCAL_ARCHIVE_DIRECTORY", "__REPLACE__LOG_DIR/hadoop/app-type/audit/archive")
-hdfs_dest_file = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.DESTINTATION_FILE", "hostname-audit.log")
-hdfs_dest_flush_int_sec = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.DESTINTATION_FLUSH_INTERVAL_SECONDS", "900")
-hdfs_dest_rollover_int_sec = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.DESTINTATION_ROLLOVER_INTERVAL_SECONDS", "86400")
-hdfs_dest_open_retry_int_sec = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.DESTINTATION_OPEN_RETRY_INTERVAL_SECONDS", "60")
-hdfs_buffer_file = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.LOCAL_BUFFER_FILE", "time:yyyyMMdd-HHmm.ss.log")
-hdfs_buffer_flush_int_sec = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.LOCAL_BUFFER_FLUSH_INTERVAL_SECONDS", "60")
-hdfs_buffer_rollover_int_sec = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.LOCAL_BUFFER_ROLLOVER_INTERVAL_SECONDS", "600")
-hdfs_archive_max_file_count = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.LOCAL_ARCHIVE_MAX_FILE_COUNT", "10")
-ssl_keystore_file = default("/configurations/ranger-storm-plugin-properties/SSL_KEYSTORE_FILE_PATH", "/etc/hadoop/conf/ranger-plugin-keystore.jks")
-ssl_keystore_password = default("/configurations/ranger-storm-plugin-properties/SSL_KEYSTORE_PASSWORD", "myKeyFilePassword")
-ssl_truststore_file = default("/configurations/ranger-storm-plugin-properties/SSL_TRUSTSTORE_FILE_PATH", "/etc/hadoop/conf/ranger-plugin-truststore.jks")
-ssl_truststore_password = default("/configurations/ranger-storm-plugin-properties/SSL_TRUSTSTORE_PASSWORD", "changeit")
-
-common_name_for_certificate = default("/configurations/ranger-storm-plugin-properties/common.name.for.certificate", "-")
-
-repo_config_username = default("/configurations/ranger-storm-plugin-properties/REPOSITORY_CONFIG_USERNAME", "hadoop")
-repo_config_password = default("/configurations/ranger-storm-plugin-properties/REPOSITORY_CONFIG_PASSWORD", "hadoop")
-storm_ui_port = config['configurations']['storm-site']['ui.port']
-
-admin_uname = default("/configurations/ranger-env/admin_username", "admin")
-admin_password = default("/configurations/ranger-env/admin_password", "admin")
-admin_uname_password = format("{admin_uname}:{admin_password}")
-
-ambari_ranger_admin = default("/configurations/ranger-env/ranger_admin_username", "amb_ranger_admin")
-ambari_ranger_password = default("/configurations/ranger-env/ranger_admin_password", "ambari123")
-policy_user = default("/configurations/ranger-storm-plugin-properties/policy_user", "storm")
-
-#For curl command in ranger plugin to get db connector
-jdk_location = config['hostLevelParams']['jdk_location']
-java_share_dir = '/usr/share/java'
-if xa_audit_db_flavor and xa_audit_db_flavor.lower() == 'mysql':
-  jdbc_symlink_name = "mysql-jdbc-driver.jar"
-  jdbc_jar_name = "mysql-connector-java.jar"
-elif xa_audit_db_flavor and xa_audit_db_flavor.lower() == 'oracle':
-  jdbc_jar_name = "ojdbc6.jar"
-  jdbc_symlink_name = "oracle-jdbc-driver.jar"
-
-downloaded_custom_connector = format("{tmp_dir}/{jdbc_jar_name}")
-
-driver_curl_source = format("{jdk_location}/{jdbc_symlink_name}")
-driver_curl_target = format("{java_share_dir}/{jdbc_jar_name}")
\ No newline at end of file
+  from params_linux import *

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_linux.py
new file mode 100644
index 0000000..abc51d5
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_linux.py
@@ -0,0 +1,198 @@
+#!/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 ambari_commons.constants import AMBARI_SUDO_BINARY
+from resource_management.libraries.functions.version import format_hdp_stack_version, compare_versions
+from resource_management.libraries.functions.default import default
+from resource_management.libraries.script import Script
+from resource_management.libraries.functions import default, format
+import status_params
+import re
+
+def get_bare_principal(normalized_principal_name):
+  """
+  Given a normalized principal name (nimbus/c6501.ambari.apache.org@EXAMPLE.COM) returns just the
+  primary component (nimbus)
+  :param normalized_principal_name: a string containing the principal name to process
+  :return: a string containing the primary component value or None if not valid
+  """
+
+  bare_principal = None
+
+  if normalized_principal_name:
+    match = re.match(r"([^/@]+)(?:/[^@])?(?:@.*)?", normalized_principal_name)
+
+    if match:
+      bare_principal = match.group(1)
+
+  return bare_principal
+
+
+# server configurations
+config = Script.get_config()
+tmp_dir = Script.get_tmp_dir()
+sudo = AMBARI_SUDO_BINARY
+
+stack_name = default("/hostLevelParams/stack_name", None)
+
+version = default("/commandParams/version", None)
+
+stack_version_unformatted = str(config['hostLevelParams']['stack_version'])
+hdp_stack_version = format_hdp_stack_version(stack_version_unformatted)
+stack_is_hdp22_or_further = hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0
+
+#hadoop params
+if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
+  rest_lib_dir = '/usr/hdp/current/storm-client/contrib/storm-rest'
+  storm_bin_dir = "/usr/hdp/current/storm-client/bin"
+  storm_lib_dir = "/usr/hdp/current/storm-client/lib"
+else:
+  rest_lib_dir = "/usr/lib/storm/contrib/storm-rest"
+  storm_bin_dir = "/usr/bin"
+  storm_lib_dir = "/usr/lib/storm/lib/"
+
+storm_user = config['configurations']['storm-env']['storm_user']
+log_dir = config['configurations']['storm-env']['storm_log_dir']
+pid_dir = status_params.pid_dir
+conf_dir = "/etc/storm/conf"
+local_dir = config['configurations']['storm-site']['storm.local.dir']
+user_group = config['configurations']['cluster-env']['user_group']
+java64_home = config['hostLevelParams']['java_home']
+jps_binary = format("{java64_home}/bin/jps")
+nimbus_port = config['configurations']['storm-site']['nimbus.thrift.port']
+nimbus_host = config['configurations']['storm-site']['nimbus.host']
+rest_api_port = "8745"
+rest_api_admin_port = "8746"
+rest_api_conf_file = format("{conf_dir}/config.yaml")
+storm_env_sh_template = config['configurations']['storm-env']['content']
+jmxremote_port = config['configurations']['storm-env']['jmxremote_port']
+
+if 'ganglia_server_host' in config['clusterHostInfo'] and \
+                len(config['clusterHostInfo']['ganglia_server_host'])>0:
+  ganglia_installed = True
+  ganglia_server = config['clusterHostInfo']['ganglia_server_host'][0]
+  ganglia_report_interval = 60
+else:
+  ganglia_installed = False
+
+security_enabled = config['configurations']['cluster-env']['security_enabled']
+
+if security_enabled:
+  _hostname_lowercase = config['hostname'].lower()
+  _storm_principal_name = config['configurations']['storm-env']['storm_principal_name']
+  storm_jaas_principal = _storm_principal_name.replace('_HOST',_hostname_lowercase)
+  storm_keytab_path = config['configurations']['storm-env']['storm_keytab']
+
+  if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
+    storm_ui_keytab_path = config['configurations']['storm-env']['storm_ui_keytab']
+    _storm_ui_jaas_principal_name = config['configurations']['storm-env']['storm_ui_principal_name']
+    storm_ui_host = default("/clusterHostInfo/storm_ui_server_hosts", [])
+    storm_ui_jaas_principal = _storm_ui_jaas_principal_name.replace('_HOST',_hostname_lowercase)
+
+    storm_bare_jaas_principal = get_bare_principal(_storm_principal_name)
+
+    _nimbus_principal_name = config['configurations']['storm-env']['nimbus_principal_name']
+    nimbus_jaas_principal = _nimbus_principal_name.replace('_HOST', _hostname_lowercase)
+    nimbus_bare_jaas_principal = get_bare_principal(_nimbus_principal_name)
+    nimbus_keytab_path = config['configurations']['storm-env']['nimbus_keytab']
+
+if stack_is_hdp22_or_further:
+  if security_enabled:
+    storm_thrift_transport = config['configurations']['storm-site']['_storm.thrift.secure.transport']
+  else:
+    storm_thrift_transport = config['configurations']['storm-site']['_storm.thrift.nonsecure.transport']
+
+ams_collector_hosts = default("/clusterHostInfo/metrics_collector_hosts", [])
+has_metric_collector = not len(ams_collector_hosts) == 0
+if has_metric_collector:
+  metric_collector_host = ams_collector_hosts[0]
+  metric_collector_report_interval = 60
+  metric_collector_app_id = "nimbus"
+metric_collector_sink_jar = "/usr/lib/storm/lib/ambari-metrics-storm-sink*.jar"
+
+# ranger host
+ranger_admin_hosts = default("/clusterHostInfo/ranger_admin_hosts", [])
+has_ranger_admin = not len(ranger_admin_hosts) == 0
+
+if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
+  # setting flag value for ranger hive plugin
+  enable_ranger_storm = False
+  ranger_plugin_enable = default("/configurations/ranger-storm-plugin-properties/ranger-storm-plugin-enabled", "no")
+  if ranger_plugin_enable.lower() == 'yes':
+    enable_ranger_storm = True
+  elif ranger_plugin_enable.lower() == 'no':
+    enable_ranger_storm = False
+
+ambari_server_hostname = config['clusterHostInfo']['ambari_server_host'][0]
+
+#ranger storm properties
+policymgr_mgr_url = default("/configurations/admin-properties/policymgr_external_url", "http://localhost:6080")
+sql_connector_jar = default("/configurations/admin-properties/SQL_CONNECTOR_JAR", "/usr/share/java/mysql-connector-java.jar")
+xa_audit_db_flavor = default("/configurations/admin-properties/DB_FLAVOR", "MYSQL")
+xa_audit_db_name = default("/configurations/admin-properties/audit_db_name", "ranger_audit")
+xa_audit_db_user = default("/configurations/admin-properties/audit_db_user", "rangerlogger")
+xa_audit_db_password = default("/configurations/admin-properties/audit_db_password", "rangerlogger")
+xa_db_host = default("/configurations/admin-properties/db_host", "localhost")
+repo_name = str(config['clusterName']) + '_storm'
+db_enabled = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.DB.IS_ENABLED", "false")
+hdfs_enabled = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.IS_ENABLED", "false")
+hdfs_dest_dir = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.DESTINATION_DIRECTORY", "hdfs://__REPLACE__NAME_NODE_HOST:8020/ranger/audit/app-type/time:yyyyMMdd")
+hdfs_buffer_dir = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.LOCAL_BUFFER_DIRECTORY", "__REPLACE__LOG_DIR/hadoop/app-type/audit")
+hdfs_archive_dir = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.LOCAL_ARCHIVE_DIRECTORY", "__REPLACE__LOG_DIR/hadoop/app-type/audit/archive")
+hdfs_dest_file = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.DESTINTATION_FILE", "hostname-audit.log")
+hdfs_dest_flush_int_sec = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.DESTINTATION_FLUSH_INTERVAL_SECONDS", "900")
+hdfs_dest_rollover_int_sec = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.DESTINTATION_ROLLOVER_INTERVAL_SECONDS", "86400")
+hdfs_dest_open_retry_int_sec = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.DESTINTATION_OPEN_RETRY_INTERVAL_SECONDS", "60")
+hdfs_buffer_file = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.LOCAL_BUFFER_FILE", "time:yyyyMMdd-HHmm.ss.log")
+hdfs_buffer_flush_int_sec = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.LOCAL_BUFFER_FLUSH_INTERVAL_SECONDS", "60")
+hdfs_buffer_rollover_int_sec = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.LOCAL_BUFFER_ROLLOVER_INTERVAL_SECONDS", "600")
+hdfs_archive_max_file_count = default("/configurations/ranger-storm-plugin-properties/XAAUDIT.HDFS.LOCAL_ARCHIVE_MAX_FILE_COUNT", "10")
+ssl_keystore_file = default("/configurations/ranger-storm-plugin-properties/SSL_KEYSTORE_FILE_PATH", "/etc/hadoop/conf/ranger-plugin-keystore.jks")
+ssl_keystore_password = default("/configurations/ranger-storm-plugin-properties/SSL_KEYSTORE_PASSWORD", "myKeyFilePassword")
+ssl_truststore_file = default("/configurations/ranger-storm-plugin-properties/SSL_TRUSTSTORE_FILE_PATH", "/etc/hadoop/conf/ranger-plugin-truststore.jks")
+ssl_truststore_password = default("/configurations/ranger-storm-plugin-properties/SSL_TRUSTSTORE_PASSWORD", "changeit")
+
+common_name_for_certificate = default("/configurations/ranger-storm-plugin-properties/common.name.for.certificate", "-")
+
+repo_config_username = default("/configurations/ranger-storm-plugin-properties/REPOSITORY_CONFIG_USERNAME", "hadoop")
+repo_config_password = default("/configurations/ranger-storm-plugin-properties/REPOSITORY_CONFIG_PASSWORD", "hadoop")
+storm_ui_port = config['configurations']['storm-site']['ui.port']
+
+admin_uname = default("/configurations/ranger-env/admin_username", "admin")
+admin_password = default("/configurations/ranger-env/admin_password", "admin")
+admin_uname_password = format("{admin_uname}:{admin_password}")
+
+ambari_ranger_admin = default("/configurations/ranger-env/ranger_admin_username", "amb_ranger_admin")
+ambari_ranger_password = default("/configurations/ranger-env/ranger_admin_password", "ambari123")
+policy_user = default("/configurations/ranger-storm-plugin-properties/policy_user", "storm")
+
+#For curl command in ranger plugin to get db connector
+jdk_location = config['hostLevelParams']['jdk_location']
+java_share_dir = '/usr/share/java'
+if xa_audit_db_flavor and xa_audit_db_flavor.lower() == 'mysql':
+  jdbc_symlink_name = "mysql-jdbc-driver.jar"
+  jdbc_jar_name = "mysql-connector-java.jar"
+elif xa_audit_db_flavor and xa_audit_db_flavor.lower() == 'oracle':
+  jdbc_jar_name = "ojdbc6.jar"
+  jdbc_symlink_name = "oracle-jdbc-driver.jar"
+
+downloaded_custom_connector = format("{tmp_dir}/{jdbc_jar_name}")
+
+driver_curl_source = format("{jdk_location}/{jdbc_symlink_name}")
+driver_curl_target = format("{java_share_dir}/{jdbc_jar_name}")
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_windows.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_windows.py b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_windows.py
new file mode 100644
index 0000000..152baa4
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_windows.py
@@ -0,0 +1,29 @@
+#!/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 status_params import *
+
+# server configurations
+config = Script.get_config()
+
+hdp_root = os.path.abspath(os.path.join(os.environ["HADOOP_HOME"],".."))
+conf_dir = os.environ["STORM_CONF_DIR"]
+storm_user = "hadoop"

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/service_check.py b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/service_check.py
index 4eec899..483c144 100644
--- a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/service_check.py
+++ b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/service_check.py
@@ -24,8 +24,25 @@ from resource_management.core.resources import File
 from resource_management.core.resources import Execute
 from resource_management.libraries.script import Script
 from resource_management.core.source import StaticFile
+from ambari_commons import OSCheck, OSConst
+from ambari_commons.os_family_impl import OsFamilyImpl
 
 class ServiceCheck(Script):
+  pass
+
+
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class ServiceCheckWindows(ServiceCheck):
+  def service_check(self, env):
+    import params
+    env.set_params(params)
+    smoke_cmd = os.path.join(params.hdp_root,"Run-SmokeTests.cmd")
+    service = "STORM"
+    Execute(format("cmd /C {smoke_cmd} {service}"), logoutput=True)
+
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class ServiceCheckDefault(ServiceCheck):
   def service_check(self, env):
     import params
     env.set_params(params)

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/status_params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/status_params.py b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/status_params.py
index 503e0e9..2c06d16 100644
--- a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/status_params.py
+++ b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/status_params.py
@@ -20,29 +20,35 @@ limitations under the License.
 from resource_management.libraries.script import Script
 from resource_management.libraries.functions import get_kinit_path
 from resource_management.libraries.functions import default, format
+from ambari_commons import OSCheck
 
 config = Script.get_config()
 
-pid_dir = config['configurations']['storm-env']['storm_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}
+if OSCheck.is_windows_family():
+  nimbus_win_service_name = "nimbus"
+  supervisor_win_service_name = "supervisor"
+  ui_win_service_name = "ui"
+else:
+  pid_dir = config['configurations']['storm-env']['storm_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}
 
-# Security related/required params
-hostname = config['hostname']
-security_enabled = config['configurations']['cluster-env']['security_enabled']
-kinit_path_local = get_kinit_path()
-tmp_dir = Script.get_tmp_dir()
-conf_dir = "/etc/storm/conf"
-storm_user = config['configurations']['storm-env']['storm_user']
-storm_ui_principal = default('/configurations/storm-env/storm_ui_principal_name', None)
-storm_ui_keytab = default('/configurations/storm-env/storm_ui_keytab', None)
+  # Security related/required params
+  hostname = config['hostname']
+  security_enabled = config['configurations']['cluster-env']['security_enabled']
+  kinit_path_local = get_kinit_path()
+  tmp_dir = Script.get_tmp_dir()
+  conf_dir = "/etc/storm/conf"
+  storm_user = config['configurations']['storm-env']['storm_user']
+  storm_ui_principal = default('/configurations/storm-env/storm_ui_principal_name', None)
+  storm_ui_keytab = default('/configurations/storm-env/storm_ui_keytab', None)

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/storm.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/storm.py b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/storm.py
index 3ae3f92..08d5b34 100644
--- a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/storm.py
+++ b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/storm.py
@@ -27,8 +27,20 @@ from resource_management.libraries.resources.template_config import TemplateConf
 from resource_management.libraries.functions.format import format
 from resource_management.core.source import Template
 from resource_management.libraries.functions import compare_versions
-from yaml_utils import yaml_config_template
+from yaml_utils import yaml_config_template, yaml_config
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from ambari_commons import OSConst
 
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def storm():
+  import params
+  yaml_config("storm.yaml",
+              conf_dir=params.conf_dir,
+              configurations=params.config['configurations']['storm-site'],
+              owner=params.storm_user
+  )
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
 def storm():
   import params
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/supervisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/supervisor.py b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/supervisor.py
index e5852a3..0005efa 100644
--- a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/supervisor.py
+++ b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/supervisor.py
@@ -24,16 +24,13 @@ from resource_management.libraries.script import Script
 from resource_management.libraries.functions import format
 from resource_management.core.resources.system import Execute
 from resource_management.libraries.functions.version import compare_versions, format_hdp_stack_version
-
 from storm import storm
 from service import service
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyImpl
 
 
 class Supervisor(Script):
-
-  def get_stack_to_component(self):
-    return {"HDP": "storm-supervisor"}
-
   def install(self, env):
     self.install_packages(env)
     self.configure(env)
@@ -43,6 +40,31 @@ class Supervisor(Script):
     env.set_params(params)
     storm()
 
+
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class SupervisorWindows(Supervisor):
+  def start(self, env):
+    import status_params
+    env.set_params(status_params)
+    self.configure(env)
+    Service(status_params.supervisor_win_service_name, action="start")
+
+  def stop(self, env):
+    import status_params
+    env.set_params(status_params)
+    Service(status_params.supervisor_win_service_name, action="stop")
+
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+    check_windows_service_status(status_params.supervisor_win_service_name)
+
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class SupervisorDefault(Supervisor):
+  def get_stack_to_component(self):
+    return {"HDP": "storm-supervisor"}
+
   def pre_rolling_restart(self, env):
     import params
     env.set_params(params)
@@ -68,7 +90,6 @@ class Supervisor(Script):
   def status(self, env):
     import status_params
     env.set_params(status_params)
-
     check_process_status(status_params.pid_supervisor)
 
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/ui_server.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/ui_server.py b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/ui_server.py
index cd0ba0a..bd09ffe 100644
--- a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/ui_server.py
+++ b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/ui_server.py
@@ -31,12 +31,11 @@ from resource_management.libraries.functions.security_commons import build_expec
   cached_kinit_executor, get_params_from_filesystem, validate_security_config_properties, \
   FILE_TYPE_JAAS_CONF
 from setup_ranger_storm import setup_ranger_storm
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyImpl
 
-class UiServer(Script):
-
-  def get_stack_to_component(self):
-    return {"HDP": "storm-client"}
 
+class UiServer(Script):
   def install(self, env):
     self.install_packages(env)
     self.configure(env)
@@ -44,13 +43,35 @@ class UiServer(Script):
   def configure(self, env):
     import params
     env.set_params(params)
-
     storm()
 
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class UiServerWindows(UiServer):
+  def start(self, env):
+    import status_params
+    env.set_params(status_params)
+    self.configure(env)
+    Service(status_params.ui_win_service_name, action="start")
+
+  def stop(self, env):
+    import status_params
+    env.set_params(status_params)
+    Service(status_params.ui_win_service_name, action="stop")
+
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+    check_windows_service_status(status_params.ui_win_service_name)
+
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class UiServerDefault(UiServer):
+  def get_stack_to_component(self):
+    return {"HDP": "storm-client"}
+
   def pre_rolling_restart(self, env):
     import params
     env.set_params(params)
-
     if params.version and compare_versions(format_hdp_stack_version(params.version), '2.2.0.0') >= 0:
       Execute(format("hdp-select set storm-client {version}"))
 
@@ -64,7 +85,6 @@ class UiServer(Script):
   def stop(self, env, rolling_restart=False):
     import params
     env.set_params(params)
-
     service("ui", action="stop")
 
   def status(self, env):

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/yaml_utils.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/yaml_utils.py b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/yaml_utils.py
index 0034123..51ad670 100644
--- a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/yaml_utils.py
+++ b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/yaml_utils.py
@@ -65,4 +65,15 @@ storm_yaml_template = """{% for key, value in configurations|dictsort if not key
 def yaml_config_template(configurations):
   return InlineTemplate(storm_yaml_template, configurations=configurations,
                         extra_imports=[escape_yaml_propetry, replace_jaas_placeholder, resource_management,
-                                       resource_management.core, resource_management.core.source])
\ No newline at end of file
+                                       resource_management.core, resource_management.core.source])
+
+def yaml_config(filename, configurations = None, conf_dir = None, owner = None, group = None):
+  import params
+  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 (os.path.join(params.conf_dir, filename),
+        content = config_content,
+        owner = owner,
+        mode = "f"
+  )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/configuration/storm-env.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/configuration/storm-env.xml b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/configuration/storm-env.xml
index 4d9aae6..3716b68 100644
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/configuration/storm-env.xml
+++ b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/configuration/storm-env.xml
@@ -29,19 +29,15 @@
   <property>
     <name>storm_log_dir</name>
     <value>c:\hadoop\logs\storm</value>
-    <description></description>
   </property>
   <property>
     <name>storm_pid_dir</name>
     <value>c:\hadoop\run\storm</value>
-    <description></description>
   </property>
 
   <!-- storm-env.cmd -->
   <property>
     <name>content</name>
-    <description>This is the jinja template for storm-env.cmd file</description>
-    <value>
-    </value>
+    <value/>
   </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/configuration/storm-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/configuration/storm-site.xml b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/configuration/storm-site.xml
index c0143f1..060ec23 100644
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/configuration/storm-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/configuration/storm-site.xml
@@ -24,16 +24,9 @@
   <property>
     <name>storm.local.dir</name>
     <value>c:\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>dev.zookeeper.path</name>
     <value>c:\hadoop\temp\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/ambari/blob/4a9f8391/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/nimbus.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/nimbus.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/nimbus.py
deleted file mode 100644
index 322ecfd..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/nimbus.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.
-
-"""
-
-import sys
-from resource_management import *
-import service_mapping
-from storm import storm
-
-
-class Nimbus(Script):
-  def install(self, env):
-    if not check_windows_service_exists(service_mapping.nimbus_win_service_name):
-      self.install_packages(env)
-    self.configure(env)
-
-  def start(self, env):
-    self.configure(env)
-    Service(service_mapping.nimbus_win_service_name, action="start")
-
-  def stop(self, env):
-    Service(service_mapping.nimbus_win_service_name, action="stop")
-
-  def configure(self, env):
-    import params
-    env.set_params(params)
-    storm()
-
-  def status(self, env):
-    check_windows_service_status(service_mapping.nimbus_win_service_name)
-    pass
-
-if __name__ == "__main__":
-  Nimbus().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/params.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/params.py
deleted file mode 100644
index 8495720..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/params.py
+++ /dev/null
@@ -1,30 +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()
-
-hdp_root = os.path.abspath(os.path.join(os.environ["HADOOP_HOME"],".."))
-conf_dir = os.environ["STORM_CONF_DIR"]
-# storm_user = config['configurations']['storm-env']['storm_user']
-storm_user = "hadoop"

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/service_check.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/service_check.py
deleted file mode 100644
index 788044a..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/service_check.py
+++ /dev/null
@@ -1,32 +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 *
-
-class ServiceCheck(Script):
-  def service_check(self, env):
-    import params
-    env.set_params(params)
-    smoke_cmd = os.path.join(params.hdp_root,"Run-SmokeTests.cmd")
-    service = "STORM"
-    Execute(format("cmd /C {smoke_cmd} {service}"), logoutput=True)
-
-if __name__ == "__main__":
-  ServiceCheck().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/service_mapping.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/service_mapping.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/service_mapping.py
deleted file mode 100644
index 254abd2..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/service_mapping.py
+++ /dev/null
@@ -1,22 +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.
-
-"""
-
-nimbus_win_service_name = "nimbus"
-supervisor_win_service_name = "supervisor"
-ui_win_service_name = "ui"

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/status_params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/status_params.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/status_params.py
deleted file mode 100644
index bd97604..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/status_params.py
+++ /dev/null
@@ -1,22 +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()

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/storm.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/storm.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/storm.py
deleted file mode 100644
index e0bc25b..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/storm.py
+++ /dev/null
@@ -1,33 +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
-
-  yaml_config("storm.yaml",
-              conf_dir=params.conf_dir,
-              configurations=params.config['configurations']['storm-site'],
-              owner=params.storm_user
-  )

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/supervisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/supervisor.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/supervisor.py
deleted file mode 100644
index 079f51f..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/supervisor.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.
-
-"""
-
-import sys
-from resource_management import *
-import service_mapping
-from storm import storm
-
-
-class Supervisor(Script):
-  def install(self, env):
-    if not check_windows_service_exists(service_mapping.supervisor_win_service_name):
-      self.install_packages(env)
-    self.configure(env)
-
-  def start(self, env):
-    self.configure(env)
-    Service(service_mapping.supervisor_win_service_name, action="start")
-
-  def stop(self, env):
-    Service(service_mapping.supervisor_win_service_name, action="stop")
-
-  def configure(self, env):
-    import params
-    env.set_params(params)
-    storm()
-
-  def status(self, env):
-    check_windows_service_status(service_mapping.supervisor_win_service_name)
-    pass
-
-if __name__ == "__main__":
-  Supervisor().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/ui_server.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/ui_server.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/ui_server.py
deleted file mode 100644
index ebd2830..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/ui_server.py
+++ /dev/null
@@ -1,51 +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 *
-import service_mapping
-from storm import storm
-
-
-
-class UiServer(Script):
-  def install(self, env):
-      if not check_windows_service_exists(service_mapping.ui_win_service_name):
-          self.install_packages(env)
-          self.configure(env)
-
-  def start(self, env):
-      self.configure(env)
-      Service(service_mapping.ui_win_service_name, action="start")
-
-  def stop(self, env):
-      Service(service_mapping.ui_win_service_name, action="stop")
-
-  def configure(self, env):
-      import params
-      env.set_params(params)
-      storm()
-
-  def status(self, env):
-      check_windows_service_status(service_mapping.ui_win_service_name)
-      pass
-
-if __name__ == "__main__":
-  UiServer().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/yaml_config.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/yaml_config.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/package/scripts/yaml_config.py
deleted file mode 100644
index a66f2a3..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/STORM/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.
-
-"""
-
-
-from resource_management import *
-import os
-
-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,
-  owner = None,
-  group = None
-):
-    import params
-    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 (os.path.join(params.conf_dir, filename),
-      content = config_content,
-      owner = owner,
-      mode = "f"
-    )

http://git-wip-us.apache.org/repos/asf/ambari/blob/4a9f8391/ambari-server/src/main/resources/stacks/HDPWIN/2.2/services/STORM/configuration/storm-site.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.2/services/STORM/configuration/storm-site.xml b/ambari-server/src/main/resources/stacks/HDPWIN/2.2/services/STORM/configuration/storm-site.xml
index 1a50e8f..480bef8 100644
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.2/services/STORM/configuration/storm-site.xml
+++ b/ambari-server/src/main/resources/stacks/HDPWIN/2.2/services/STORM/configuration/storm-site.xml
@@ -25,13 +25,14 @@
   <property>
     <name>storm.thrift.transport</name>
     <value>{{storm_thrift_transport}}</value>
-    <description>The transport plug-in that used for Thrift client/server communication.</description>
   </property>
+
   <property>
     <name>_storm.thrift.nonsecure.transport</name>
     <value>backtype.storm.security.auth.SimpleTransportPlugin</value>
     <description>The transport plug-in that used for non-secure mode for for Thrift client/server communication.</description>
   </property>
+
   <property>
     <name>_storm.thrift.secure.transport</name>
     <value>backtype.storm.security.auth.kerberos.KerberosSaslTransportPlugin</value>
@@ -41,40 +42,31 @@
   <property>
     <name>java.library.path</name>
     <value>/usr/local/lib:/opt/local/lib:/usr/lib:/usr/hdp/current/storm-client/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>nimbus.childopts</name>
     <value>-Xmx1024m _JAAS_PLACEHOLDER -javaagent:/usr/hdp/current/storm-nimbus/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=localhost,port=8649,wireformat31x=true,mode=multicast,config=/usr/hdp/current/storm-nimbus/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>worker.childopts</name>
     <value>-Xmx768m _JAAS_PLACEHOLDER -javaagent:/usr/hdp/current/storm-client/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=localhost,port=8650,wireformat31x=true,mode=multicast,config=/usr/hdp/current/storm-client/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>ui.childopts</name>
     <value>-Xmx768m _JAAS_PLACEHOLDER</value>
-    <description>Childopts for Storm UI Java process.</description>
   </property>
 
   <property>
     <name>supervisor.childopts</name>
     <value>-Xmx256m _JAAS_PLACEHOLDER -Dcom.sun.management.jmxremote -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false -Dcom.sun.management.jmxremote.port=56431 -javaagent:/usr/hdp/current/storm-supervisor/contrib/storm-jmxetric/lib/jmxetric-1.0.4.jar=host=localhost,port=8650,wireformat31x=true,mode=multicast,config=/usr/hdp/current/storm-supervisor/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>logviewer.childopts</name>
     <value>-Xmx128m _JAAS_PLACEHOLDER</value>
-    <description>Childopts for log viewer java process.</description>
   </property>
   
   <property>