You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by jl...@apache.org on 2015/03/27 17:03:07 UTC

[1/4] ambari git commit: AMBARI-10224: [WinTP2] Merge HDPWIN ZOOKEEPER package scripts to common services (jluniya)

Repository: ambari
Updated Branches:
  refs/heads/trunk 8073405a4 -> 391cc7a6f


AMBARI-10224: [WinTP2] Merge HDPWIN ZOOKEEPER package scripts to common services (jluniya)


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

Branch: refs/heads/trunk
Commit: e4686db52f7bc624634f67524c961cfee89296bc
Parents: 8073405
Author: Jayush Luniya <jl...@hortonworks.com>
Authored: Fri Mar 27 08:49:55 2015 -0700
Committer: Jayush Luniya <jl...@hortonworks.com>
Committed: Fri Mar 27 08:49:55 2015 -0700

----------------------------------------------------------------------
 .../3.4.5.2.0/package/scripts/params.py         | 79 +---------------
 .../3.4.5.2.0/package/scripts/params_linux.py   | 98 ++++++++++++++++++++
 .../3.4.5.2.0/package/scripts/params_windows.py | 61 ++++++++++++
 .../3.4.5.2.0/package/scripts/service_check.py  | 16 ++++
 .../3.4.5.2.0/package/scripts/status_params.py  | 22 +++--
 .../3.4.5.2.0/package/scripts/zookeeper.py      | 53 ++++++++---
 .../package/scripts/zookeeper_client.py         | 45 +++++----
 .../package/scripts/zookeeper_server.py         | 53 +++++++----
 .../package/scripts/zookeeper_service.py        | 11 +++
 .../ZOOKEEPER/package/scripts/params.py         | 59 ------------
 .../ZOOKEEPER/package/scripts/service_check.py  | 34 -------
 .../package/scripts/service_mapping.py          | 22 -----
 .../ZOOKEEPER/package/scripts/zookeeper.py      | 63 -------------
 .../package/scripts/zookeeper_client.py         | 43 ---------
 .../package/scripts/zookeeper_server.py         | 51 ----------
 .../package/templates/configuration.xsl.j2      | 42 ---------
 .../ZOOKEEPER/package/templates/zoo.cfg.j2      | 69 --------------
 .../2.0.6/ZOOKEEPER/test_zookeeper_client.py    |  8 +-
 .../2.0.6/ZOOKEEPER/test_zookeeper_server.py    |  7 ++
 .../src/test/python/stacks/utils/RMFTestCase.py |  6 ++
 20 files changed, 329 insertions(+), 513 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/params.py b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/params.py
index 4d48399..fbefec9 100644
--- a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/params.py
@@ -18,81 +18,10 @@ limitations under the License.
 Ambari Agent
 
 """
+from ambari_commons import OSCheck
 
-from resource_management.libraries.functions.version import format_hdp_stack_version, compare_versions
-from resource_management.libraries.functions.default import default
-from resource_management import *
-import status_params
-
-# server configurations
-config = Script.get_config()
-tmp_dir = Script.get_tmp_dir()
-
-stack_version_unformatted = str(config['hostLevelParams']['stack_version'])
-hdp_stack_version = format_hdp_stack_version(stack_version_unformatted)
-
-stack_name = default("/hostLevelParams/stack_name", None)
-
-# New Cluster Stack Version that is defined during the RESTART of a Rolling Upgrade
-version = default("/commandParams/version", None)
-
-#hadoop params
-if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
-  role_root = "zookeeper-client"
-  command_role = default("/role", "")
-
-  if command_role == "ZOOKEEPER_SERVER":
-    role_root = "zookeeper-server"
-
-  zk_home = format("/usr/hdp/current/{role_root}")
-  zk_bin = format("/usr/hdp/current/{role_root}/bin")
-  zk_cli_shell = format("/usr/hdp/current/{role_root}/bin/zkCli.sh")
+if OSCheck.is_windows_family():
+  from params_windows import *
 else:
-  zk_home = "/usr"
-  zk_bin = "/usr/lib/zookeeper/bin"
-  zk_cli_shell = "/usr/lib/zookeeper/bin/zkCli.sh"
-
-
-config_dir = "/etc/zookeeper/conf"
-zk_user =  config['configurations']['zookeeper-env']['zk_user']
-hostname = config['hostname']
-user_group = config['configurations']['cluster-env']['user_group']
-zk_env_sh_template = config['configurations']['zookeeper-env']['content']
-
-zk_log_dir = config['configurations']['zookeeper-env']['zk_log_dir']
-zk_data_dir = config['configurations']['zoo.cfg']['dataDir']
-zk_pid_dir = status_params.zk_pid_dir
-zk_pid_file = status_params.zk_pid_file
-zk_server_heapsize = "-Xmx1024m"
+  from params_linux import *
 
-client_port = default('/configurations/zoo.cfg/clientPort', None)
-
-if 'zoo.cfg' in config['configurations']:
-  zoo_cfg_properties_map = config['configurations']['zoo.cfg']
-else:
-  zoo_cfg_properties_map = {}
-zoo_cfg_properties_map_length = len(zoo_cfg_properties_map)
-
-zk_principal_name = default("/configurations/zookeeper-env/zookeeper_principal_name", "zookeeper@EXAMPLE.COM")
-zk_principal = zk_principal_name.replace('_HOST',hostname.lower())
-
-java64_home = config['hostLevelParams']['java_home']
-
-zookeeper_hosts = config['clusterHostInfo']['zookeeper_hosts']
-zookeeper_hosts.sort()
-
-zk_keytab_path = config['configurations']['zookeeper-env']['zookeeper_keytab_path']
-zk_server_jaas_file = format("{config_dir}/zookeeper_jaas.conf")
-zk_client_jaas_file = format("{config_dir}/zookeeper_client_jaas.conf")
-security_enabled = config['configurations']['cluster-env']['security_enabled']
-
-smoke_user_keytab = config['configurations']['cluster-env']['smokeuser_keytab']
-smokeuser = config['configurations']['cluster-env']['smokeuser']
-smokeuser_principal = config['configurations']['cluster-env']['smokeuser_principal_name']
-kinit_path_local = functions.get_kinit_path()
-
-#log4j.properties
-if (('zookeeper-log4j' in config['configurations']) and ('content' in config['configurations']['zookeeper-log4j'])):
-  log4j_props = config['configurations']['zookeeper-log4j']['content']
-else:
-  log4j_props = None

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/params_linux.py
new file mode 100644
index 0000000..5d3d9d1
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/params_linux.py
@@ -0,0 +1,98 @@
+"""
+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.
+
+Ambari Agent
+
+"""
+
+from resource_management.libraries.functions.version import format_hdp_stack_version, compare_versions
+from resource_management.libraries.functions.default import default
+from resource_management import *
+import status_params
+
+# server configurations
+config = Script.get_config()
+tmp_dir = Script.get_tmp_dir()
+
+stack_version_unformatted = str(config['hostLevelParams']['stack_version'])
+hdp_stack_version = format_hdp_stack_version(stack_version_unformatted)
+
+stack_name = default("/hostLevelParams/stack_name", None)
+
+# New Cluster Stack Version that is defined during the RESTART of a Rolling Upgrade
+version = default("/commandParams/version", None)
+
+#hadoop params
+if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
+  role_root = "zookeeper-client"
+  command_role = default("/role", "")
+
+  if command_role == "ZOOKEEPER_SERVER":
+    role_root = "zookeeper-server"
+
+  zk_home = format("/usr/hdp/current/{role_root}")
+  zk_bin = format("/usr/hdp/current/{role_root}/bin")
+  zk_cli_shell = format("/usr/hdp/current/{role_root}/bin/zkCli.sh")
+else:
+  zk_home = "/usr"
+  zk_bin = "/usr/lib/zookeeper/bin"
+  zk_cli_shell = "/usr/lib/zookeeper/bin/zkCli.sh"
+
+
+config_dir = "/etc/zookeeper/conf"
+zk_user =  config['configurations']['zookeeper-env']['zk_user']
+hostname = config['hostname']
+user_group = config['configurations']['cluster-env']['user_group']
+zk_env_sh_template = config['configurations']['zookeeper-env']['content']
+
+zk_log_dir = config['configurations']['zookeeper-env']['zk_log_dir']
+zk_data_dir = config['configurations']['zoo.cfg']['dataDir']
+zk_pid_dir = status_params.zk_pid_dir
+zk_pid_file = status_params.zk_pid_file
+zk_server_heapsize = "-Xmx1024m"
+
+client_port = default('/configurations/zoo.cfg/clientPort', None)
+
+if 'zoo.cfg' in config['configurations']:
+  zoo_cfg_properties_map = config['configurations']['zoo.cfg']
+else:
+  zoo_cfg_properties_map = {}
+zoo_cfg_properties_map_length = len(zoo_cfg_properties_map)
+
+zk_principal_name = default("/configurations/zookeeper-env/zookeeper_principal_name", "zookeeper@EXAMPLE.COM")
+zk_principal = zk_principal_name.replace('_HOST',hostname.lower())
+
+java64_home = config['hostLevelParams']['java_home']
+
+zookeeper_hosts = config['clusterHostInfo']['zookeeper_hosts']
+zookeeper_hosts.sort()
+
+zk_keytab_path = config['configurations']['zookeeper-env']['zookeeper_keytab_path']
+zk_server_jaas_file = format("{config_dir}/zookeeper_jaas.conf")
+zk_client_jaas_file = format("{config_dir}/zookeeper_client_jaas.conf")
+security_enabled = config['configurations']['cluster-env']['security_enabled']
+
+smoke_user_keytab = config['configurations']['cluster-env']['smokeuser_keytab']
+smokeuser = config['configurations']['cluster-env']['smokeuser']
+smokeuser_principal = config['configurations']['cluster-env']['smokeuser_principal_name']
+kinit_path_local = functions.get_kinit_path()
+
+#log4j.properties
+if (('zookeeper-log4j' in config['configurations']) and ('content' in config['configurations']['zookeeper-log4j'])):
+  log4j_props = config['configurations']['zookeeper-log4j']['content']
+else:
+  log4j_props = None
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/params_windows.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/params_windows.py b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/params_windows.py
new file mode 100644
index 0000000..c7302a5
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/params_windows.py
@@ -0,0 +1,61 @@
+"""
+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.
+
+Ambari Agent
+
+"""
+from ambari_commons.str_utils import ensure_double_backslashes
+from resource_management import *
+import status_params
+
+# server configurations
+config = Script.get_config()
+
+# notused zookeeper_home_dir = os.environ["ZOOKEEPER_HOME"]
+config_dir = os.environ["ZOOKEEPER_CONF_DIR"]
+hdp_root = os.environ["HADOOP_NODE_INSTALL_ROOT"]
+zk_user = "hadoop"
+
+# notused zk_log_dir = config['configurations']['zookeeper-env']['zk_log_dir']
+zk_data_dir = ensure_double_backslashes(config['configurations']['zoo.cfg']['dataDir'])
+tickTime = config['configurations']['zoo.cfg']['tickTime']
+initLimit = config['configurations']['zoo.cfg']['initLimit']
+syncLimit = config['configurations']['zoo.cfg']['syncLimit']
+clientPort = config['configurations']['zoo.cfg']['clientPort']
+
+if 'zoo.cfg' in config['configurations']:
+  zoo_cfg_properties_map = config['configurations']['zoo.cfg'].copy()
+  # Fix the data dir - ZK won't start unless the backslashes are doubled
+  zoo_cfg_properties_map['dataDir'] = zk_data_dir
+else:
+  zoo_cfg_properties_map = {}
+zoo_cfg_properties_map_length = len(zoo_cfg_properties_map)
+
+zookeeper_hosts = config['clusterHostInfo']['zookeeper_hosts']
+zookeeper_hosts.sort()
+hostname = config['hostname']
+
+_authentication = config['configurations']['core-site']['hadoop.security.authentication']
+security_enabled = ( not is_empty(_authentication) and _authentication == 'kerberos')
+user_group = None
+zookeeper_win_service_name = status_params.zookeeper_win_service_name
+
+#log4j.properties
+if (('zookeeper-log4j' in config['configurations']) and ('content' in config['configurations']['zookeeper-log4j'])):
+  log4j_props = config['configurations']['zookeeper-log4j']['content']
+else:
+  log4j_props = None
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/service_check.py b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/service_check.py
index 4b3f8d4..6ffe5f5 100644
--- a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/service_check.py
+++ b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/service_check.py
@@ -20,8 +20,14 @@ Ambari Agent
 """
 
 from resource_management import *
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
 
 class ZookeeperServiceCheck(Script):
+  pass
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class ZookeeperServiceCheckLinux(ZookeeperServiceCheck):
   def service_check(self, env):
     import params
     env.set_params(params)
@@ -49,5 +55,15 @@ class ZookeeperServiceCheck(Script):
             logoutput=True
     )
 
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class ZookeeperServiceCheckWindows(ZookeeperServiceCheck):
+  def service_check(self, env):
+    import params
+    env.set_params(params)
+
+    smoke_cmd = os.path.join(params.hdp_root,"Run-SmokeTests.cmd")
+    service = "Zookeeper"
+    Execute(format("cmd /C {smoke_cmd} {service}"), logoutput=True)
+
 if __name__ == "__main__":
   ZookeeperServiceCheck().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/status_params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/status_params.py b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/status_params.py
index 55d0a30..5b2c953 100644
--- a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/status_params.py
+++ b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/status_params.py
@@ -19,16 +19,20 @@ limitations under the License.
 """
 
 from resource_management import *
+from ambari_commons import OSCheck
 
 config = Script.get_config()
 
-zk_pid_dir = config['configurations']['zookeeper-env']['zk_pid_dir']
-zk_pid_file = format("{zk_pid_dir}/zookeeper_server.pid")
+if OSCheck.is_windows_family():
+  zookeeper_win_service_name = "zkServer"
+else:
+  zk_pid_dir = config['configurations']['zookeeper-env']['zk_pid_dir']
+  zk_pid_file = format("{zk_pid_dir}/zookeeper_server.pid")
 
-# Security related/required params
-hostname = config['hostname']
-security_enabled = config['configurations']['cluster-env']['security_enabled']
-kinit_path_local = functions.get_kinit_path()
-tmp_dir = Script.get_tmp_dir()
-config_dir = "/etc/zookeeper/conf"
-zk_user =  config['configurations']['zookeeper-env']['zk_user']
+  # Security related/required params
+  hostname = config['hostname']
+  security_enabled = config['configurations']['cluster-env']['security_enabled']
+  kinit_path_local = functions.get_kinit_path()
+  tmp_dir = Script.get_tmp_dir()
+  config_dir = "/etc/zookeeper/conf"
+  zk_user =  config['configurations']['zookeeper-env']['zk_user']

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper.py b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper.py
index aa1c891..4bd94bc 100644
--- a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper.py
+++ b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper.py
@@ -22,8 +22,10 @@ import os
 
 from resource_management import *
 import sys
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
 
-
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
 def zookeeper(type = None):
   import params
 
@@ -33,7 +35,7 @@ def zookeeper(type = None):
             group=params.user_group
   )
 
-  File(format("{config_dir}/zookeeper-env.sh"),
+  File(os.path.join(params.config_dir, "zookeeper-env.sh"),
        content=InlineTemplate(params.zk_env_sh_template),
        owner=params.zk_user,
        group=params.user_group
@@ -65,20 +67,20 @@ def zookeeper(type = None):
   if type == 'server':
     myid = str(sorted(params.zookeeper_hosts).index(params.hostname) + 1)
 
-    File(format("{zk_data_dir}/myid"),
+    File(os.path.join(params.zk_data_dir, "myid"),
          mode = 0644,
          content = myid
     )
 
   if (params.log4j_props != None):
-    File(format("{params.config_dir}/log4j.properties"),
+    File(os.path.join(params.config_dir, "log4j.properties"),
          mode=0644,
          group=params.user_group,
          owner=params.zk_user,
          content=params.log4j_props
     )
-  elif (os.path.exists(format("{params.config_dir}/log4j.properties"))):
-    File(format("{params.config_dir}/log4j.properties"),
+  elif (os.path.exists(os.path.join(params.config_dir, "log4j.properties"))):
+    File(os.path.join(params.config_dir, "log4j.properties"),
          mode=0644,
          group=params.user_group,
          owner=params.zk_user
@@ -91,21 +93,50 @@ def zookeeper(type = None):
     else:
       configFile("zookeeper_client_jaas.conf", template_name="zookeeper_client_jaas.conf.j2")
 
-  File(format("{config_dir}/zoo_sample.cfg"),
+  File(os.path.join(params.config_dir, "zoo_sample.cfg"),
        owner=params.zk_user,
        group=params.user_group
   )
 
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def zookeeper(type = None):
+  import params
+  configFile("zoo.cfg", template_name="zoo.cfg.j2", mode="f")
+  configFile("configuration.xsl", template_name="configuration.xsl.j2", mode="f")
+
+  Directory(params.zk_data_dir,
+            owner=params.zk_user,
+            mode="(OI)(CI)F",
+            recursive=True
+  )
+  if (params.log4j_props != None):
+    File(os.path.join(params.config_dir, "log4j.properties"),
+         mode="f",
+         owner=params.zk_user,
+         content=params.log4j_props
+    )
+  elif (os.path.exists(os.path.join(params.config_dir, "log4j.properties"))):
+    File(os.path.join(params.config_dir, "log4j.properties"),
+         mode="f",
+         owner=params.zk_user
+    )
+  if type == 'server':
+    myid = str(sorted(params.zookeeper_hosts).index(params.hostname) + 1)
+    File(os.path.join(params.zk_data_dir, "myid"),
+         owner=params.zk_user,
+         mode = "f",
+         content = myid
+    )
 
-def configFile(name, template_name=None):
+def configFile(name, template_name=None, mode=None):
   import params
 
-  File(format("{config_dir}/{name}"),
+  File(os.path.join(params.config_dir, name),
        content=Template(template_name),
        owner=params.zk_user,
-       group=params.user_group
+       group=params.user_group,
+       mode=mode
   )
 
 
 
-

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_client.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_client.py b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_client.py
index dcaaab6..92264ab 100644
--- a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_client.py
+++ b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_client.py
@@ -23,31 +23,17 @@ import sys
 from resource_management import *
 from resource_management.libraries.functions.version import compare_versions, format_hdp_stack_version
 from resource_management.libraries.functions.format import format
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
 
 from zookeeper import zookeeper
 
 class ZookeeperClient(Script):
-
-  def get_stack_to_component(self):
-    return {"HDP": "zookeeper-client"}
-
-  def install(self, env):
-    self.install_packages(env)
-    self.configure(env)
-
   def configure(self, env):
     import params
     env.set_params(params)
-
     zookeeper(type='client')
-
-  def pre_rolling_restart(self, env):
-    Logger.info("Executing Rolling Upgrade pre-restart")
-    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 zookeeper-client {version}"))
+    pass
 
   def start(self, env, rolling_restart=False):
     import params
@@ -63,5 +49,30 @@ class ZookeeperClient(Script):
   def status(self, env):
     raise ClientComponentHasNoStatus()
 
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class ZookeeperClientLinux(ZookeeperClient):
+  def get_stack_to_component(self):
+    return {"HDP": "zookeeper-client"}
+
+  def install(self, env):
+    self.install_packages(env)
+    self.configure(env)
+
+  def pre_rolling_restart(self, env):
+    Logger.info("Executing Rolling Upgrade pre-restart")
+    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 zookeeper-client {version}"))
+
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class ZookeeperClientWindows(ZookeeperClient):
+  def install(self, env):
+    # client checks env var to determine if it is installed
+    if not os.environ.has_key("ZOOKEEPER_HOME"):
+      self.install_packages(env)
+    self.configure(env)
+
 if __name__ == "__main__":
   ZookeeperClient().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_server.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_server.py b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_server.py
index 72a8f0b..b2946f4 100644
--- a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_server.py
+++ b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_server.py
@@ -19,6 +19,7 @@ Ambari Agent
 
 """
 import random
+import sys
 
 from resource_management.libraries.script.script import Script
 from resource_management.libraries.functions import get_unique_id_and_date
@@ -34,10 +35,31 @@ from resource_management.libraries.functions.format import format
 from resource_management.libraries.functions.validate import call_and_match_output
 from zookeeper import zookeeper
 from zookeeper_service import zookeeper_service
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyImpl
 
 
 class ZookeeperServer(Script):
 
+  def configure(self, env):
+    import params
+    env.set_params(params)
+    zookeeper(type='server')
+
+  def start(self, env, rolling_restart=False):
+    import params
+    env.set_params(params)
+    self.configure(env)
+    zookeeper_service(action = 'start')
+
+  def stop(self, env, rolling_restart=False):
+    import params
+    env.set_params(params)
+    zookeeper_service(action = 'stop')
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class ZookeeperServerLinux(ZookeeperServer):
+
   def get_stack_to_component(self):
     return {"HDP": "zookeeper-server"}
 
@@ -45,11 +67,6 @@ class ZookeeperServer(Script):
     self.install_packages(env)
     self.configure(env)
 
-  def configure(self, env):
-    import params
-    env.set_params(params)
-    zookeeper(type='server')
-
   def pre_rolling_restart(self, env):
     Logger.info("Executing Rolling Upgrade pre-restart")
     import params
@@ -58,12 +75,6 @@ class ZookeeperServer(Script):
     if params.version and compare_versions(format_hdp_stack_version(params.version), '2.2.0.0') >= 0:
       Execute(format("hdp-select set zookeeper-server {version}"))
 
-  def start(self, env, rolling_restart=False):
-    import params
-    env.set_params(params)
-    self.configure(env)
-    zookeeper_service(action = 'start')
-
   def post_rolling_restart(self, env):
     Logger.info("Executing Rolling Upgrade post-restart")
     import params
@@ -87,11 +98,6 @@ class ZookeeperServer(Script):
       if code == 0 and out:
         Logger.info(out)
 
-  def stop(self, env, rolling_restart=False):
-    import params
-    env.set_params(params)
-    zookeeper_service(action = 'stop')
-
   def status(self, env):
     import status_params
     env.set_params(status_params)
@@ -99,7 +105,6 @@ class ZookeeperServer(Script):
 
   def security_status(self, env):
     import status_params
-
     env.set_params(status_params)
 
     if status_params.security_enabled:
@@ -150,5 +155,19 @@ class ZookeeperServer(Script):
       self.put_structured_out({"securityState": "UNSECURED"})
 
 
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class ZookeeperServerWindows(ZookeeperServer):
+  def install(self, env):
+    from resource_management.libraries.functions.windows_service_utils import check_windows_service_exists
+    import params
+    if not check_windows_service_exists(params.zookeeper_win_service_name):
+      self.install_packages(env)
+    self.configure(env)
+
+  def status(self, env):
+    from resource_management.libraries.functions.windows_service_utils import check_windows_service_status
+    import status_params
+    check_windows_service_status(status_params.zookeeper_win_service_name)
+
 if __name__ == "__main__":
   ZookeeperServer().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_service.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_service.py b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_service.py
index 2be70cc..59ad417 100644
--- a/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_service.py
+++ b/ambari-server/src/main/resources/common-services/ZOOKEEPER/3.4.5.2.0/package/scripts/zookeeper_service.py
@@ -20,7 +20,10 @@ Ambari Agent
 """
 
 from resource_management import *
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
 
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
 def zookeeper_service(action='start'):
   import params
 
@@ -48,3 +51,11 @@ def zookeeper_service(action='start'):
             user=params.zk_user
     )
     Execute(rm_pid)
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def zookeeper_service(action='start'):
+  import params
+  if action == 'start':
+    Service(params.zookeeper_win_service_name, action="start")
+  elif action == 'stop':
+    Service(params.zookeeper_win_service_name, action="stop")
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/params.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/params.py
deleted file mode 100644
index de5b9bb..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/params.py
+++ /dev/null
@@ -1,59 +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.
-
-Ambari Agent
-
-"""
-from ambari_commons.str_utils import ensure_double_backslashes
-
-from resource_management import *
-
-# server configurations
-config = Script.get_config()
-
-# notused zookeeper_home_dir = os.environ["ZOOKEEPER_HOME"]
-zookeeper_conf_dir = os.environ["ZOOKEEPER_CONF_DIR"]
-hdp_root = os.environ["HADOOP_NODE_INSTALL_ROOT"]
-zk_user = "hadoop"
-
-# notused zk_log_dir = config['configurations']['zookeeper-env']['zk_log_dir']
-zk_data_dir = ensure_double_backslashes(config['configurations']['zoo.cfg']['dataDir'])
-tickTime = config['configurations']['zoo.cfg']['tickTime']
-initLimit = config['configurations']['zoo.cfg']['initLimit']
-syncLimit = config['configurations']['zoo.cfg']['syncLimit']
-clientPort = config['configurations']['zoo.cfg']['clientPort']
-
-if 'zoo.cfg' in config['configurations']:
-  zoo_cfg_properties_map = config['configurations']['zoo.cfg'].copy()
-  # Fix the data dir - ZK won't start unless the backslashes are doubled
-  zoo_cfg_properties_map['dataDir'] = zk_data_dir
-else:
-  zoo_cfg_properties_map = {}
-zoo_cfg_properties_map_length = len(zoo_cfg_properties_map)
-
-zookeeper_hosts = config['clusterHostInfo']['zookeeper_hosts']
-zookeeper_hosts.sort()
-hostname = config['hostname']
-
-_authentication = config['configurations']['core-site']['hadoop.security.authentication']
-security_enabled = ( not is_empty(_authentication) and _authentication == 'kerberos')
-
-#log4j.properties
-if (('zookeeper-log4j' in config['configurations']) and ('content' in config['configurations']['zookeeper-log4j'])):
-  log4j_props = config['configurations']['zookeeper-log4j']['content']
-else:
-  log4j_props = None

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

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/service_mapping.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/service_mapping.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/service_mapping.py
deleted file mode 100644
index 5fc6da2..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/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.
-
-Ambari Agent
-
-"""
-
-zookeeper_win_service_name = "zkServer"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/zookeeper.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/zookeeper.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/zookeeper.py
deleted file mode 100644
index 2cacd57..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/zookeeper.py
+++ /dev/null
@@ -1,63 +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.
-
-Ambari Agent
-
-"""
-import os
-
-from resource_management import *
-import sys
-
-
-def zookeeper(type = None):
-  import params
-  configFile("zoo.cfg", template_name="zoo.cfg.j2")
-  configFile("configuration.xsl", template_name="configuration.xsl.j2")
-
-  Directory(params.zk_data_dir,
-            owner=params.zk_user,
-            mode="(OI)(CI)F",
-            recursive=True
-  )
-  if (params.log4j_props != None):
-    File(os.path.join(params.zookeeper_conf_dir, "log4j.properties"),
-         mode="f",
-         owner=params.zk_user,
-         content=params.log4j_props
-    )
-  elif (os.path.exists(os.path.join(params.zookeeper_conf_dir, "log4j.properties"))):
-    File(format("{params.zookeeper_conf_dir}/log4j.properties"),
-         mode="f",
-         owner=params.zk_user
-    )
-  if type == 'server':
-    myid = str(sorted(params.zookeeper_hosts).index(params.hostname) + 1)
-    File(format("{zk_data_dir}/myid"),
-         owner=params.zk_user,
-         mode = "f",
-         content = myid
-    )
-
-def configFile(name, template_name=None):
-  import params
-
-  File(os.path.join(params.zookeeper_conf_dir, name),
-       content=Template(template_name),
-       owner=params.zk_user,
-       mode="f"
-  )

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/zookeeper_client.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/zookeeper_client.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/zookeeper_client.py
deleted file mode 100644
index 663766f..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/zookeeper_client.py
+++ /dev/null
@@ -1,43 +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.
-
-Ambari Agent
-
-"""
-
-import sys
-from resource_management import *
-from zookeeper import zookeeper
-import os
-
-class ZookeeperClient(Script):
-  def install(self, env):
-    # client checks env var to determine if it is installed
-    if not os.environ.has_key("ZOOKEEPER_HOME"):
-      self.install_packages(env)
-    self.configure(env)
-
-  def configure(self, env):
-    import params
-    env.set_params(params)
-    zookeeper(type='client')
-
-  def status(self, env):
-    raise ClientComponentHasNoStatus()
-
-if __name__ == "__main__":
-  ZookeeperClient().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/zookeeper_server.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/zookeeper_server.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/zookeeper_server.py
deleted file mode 100644
index d2ad45d..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/scripts/zookeeper_server.py
+++ /dev/null
@@ -1,51 +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.
-
-Ambari Agent
-
-"""
-
-import sys
-from resource_management import *
-from zookeeper import zookeeper
-import service_mapping
-
-class ZookeeperServer(Script):
-  def install(self, env):
-    if not check_windows_service_exists(service_mapping.zookeeper_win_service_name):
-      self.install_packages(env)
-    self.configure(env)
-
-  def configure(self, env):
-    import params
-    env.set_params(params)
-    zookeeper(type='server')
-
-  def start(self, env):
-    import params
-    self.configure(env)
-    Service(service_mapping.zookeeper_win_service_name, action="start")
-
-  def stop(self, env):
-    import params
-    Service(service_mapping.zookeeper_win_service_name, action="stop")
-
-  def status(self, env):
-    check_windows_service_status(service_mapping.zookeeper_win_service_name)
-
-if __name__ == "__main__":
-  ZookeeperServer().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/templates/configuration.xsl.j2
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/templates/configuration.xsl.j2 b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/templates/configuration.xsl.j2
deleted file mode 100644
index 8830c45..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/templates/configuration.xsl.j2
+++ /dev/null
@@ -1,42 +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.
-#}
-
-<?xml version="1.0"?>
-<xsl:stylesheet xmlns:xsl="http://www.w3.org/1999/XSL/Transform" version="1.0">
-<xsl:output method="html"/>
-<xsl:template match="configuration">
-<html>
-<body>
-<table border="1">
-<tr>
- <td>name</td>
- <td>value</td>
- <td>description</td>
-</tr>
-<xsl:for-each select="property">
-  <tr>
-     <td><a name="{name}"><xsl:value-of select="name"/></a></td>
-     <td><xsl:value-of select="value"/></td>
-     <td><xsl:value-of select="description"/></td>
-  </tr>
-</xsl:for-each>
-</table>
-</body>
-</html>
-</xsl:template>
-</xsl:stylesheet>

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/templates/zoo.cfg.j2
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/templates/zoo.cfg.j2 b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/templates/zoo.cfg.j2
deleted file mode 100644
index beb4730..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/ZOOKEEPER/package/templates/zoo.cfg.j2
+++ /dev/null
@@ -1,69 +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.
-#}
-
-#
-#
-# 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.
-#
-#
-#
-
-# The number of milliseconds of each tick
-tickTime={{tickTime}}
-# The number of ticks that the initial
-# synchronization phase can take
-initLimit={{initLimit}}
-# The number of ticks that can pass between
-# sending a request and getting an acknowledgement
-syncLimit={{syncLimit}}
-# the directory where the snapshot is stored.
-dataDir={{zk_data_dir}}
-# the port at which the clients will connect
-clientPort={{clientPort}}
-{% for host in zookeeper_hosts %}
-server.{{loop.index}}={{host}}:2888:3888
-{% endfor %}
-
-{% if security_enabled %}
-authProvider.1=org.apache.zookeeper.server.auth.SASLAuthenticationProvider
-jaasLoginRenew=3600000
-kerberos.removeHostFromPrincipal=true
-kerberos.removeRealmFromPrincipal=true
-{% endif %}
-
-{% if zoo_cfg_properties_map_length > 0 %}
-# Custom properties
-{% endif %}
-{% for key, value in zoo_cfg_properties_map.iteritems() %}
-{{key}}={{value}}
-{% endfor %}

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/test/python/stacks/2.0.6/ZOOKEEPER/test_zookeeper_client.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.0.6/ZOOKEEPER/test_zookeeper_client.py b/ambari-server/src/test/python/stacks/2.0.6/ZOOKEEPER/test_zookeeper_client.py
index 68ab79e..bebec4f 100644
--- a/ambari-server/src/test/python/stacks/2.0.6/ZOOKEEPER/test_zookeeper_client.py
+++ b/ambari-server/src/test/python/stacks/2.0.6/ZOOKEEPER/test_zookeeper_client.py
@@ -21,6 +21,7 @@ from mock.mock import MagicMock, call, patch
 from stacks.utils.RMFTestCase import *
 
 @patch("os.path.exists", new = MagicMock(return_value=True))
+@patch("platform.linux_distribution", new = MagicMock(return_value="Linux"))
 class TestZookeeperClient(RMFTestCase):
   COMMON_SERVICES_PACKAGE_DIR = "ZOOKEEPER/3.4.5.2.0/package"
   STACK_VERSION = "2.0.6"
@@ -48,11 +49,13 @@ class TestZookeeperClient(RMFTestCase):
       owner = 'zookeeper',
       content = Template('zoo.cfg.j2'),
       group = 'hadoop',
+      mode = None,
     )
     self.assertResourceCalled('File', '/etc/zookeeper/conf/configuration.xsl',
       owner = 'zookeeper',
       content = Template('configuration.xsl.j2'),
       group = 'hadoop',
+      mode = None,
     )
     self.assertResourceCalled('Directory', '/var/run/zookeeper',
       owner = 'zookeeper',
@@ -106,11 +109,13 @@ class TestZookeeperClient(RMFTestCase):
       owner = 'zookeeper',
       content = Template('zoo.cfg.j2'),
       group = 'hadoop',
+      mode = None,
     )
     self.assertResourceCalled('File', '/etc/zookeeper/conf/configuration.xsl',
       owner = 'zookeeper',
       content = Template('configuration.xsl.j2'),
       group = 'hadoop',
+      mode = None,
     )
     self.assertResourceCalled('Directory', '/var/run/zookeeper',
       owner = 'zookeeper',
@@ -139,9 +144,10 @@ class TestZookeeperClient(RMFTestCase):
       owner = 'zookeeper',
       content = Template('zookeeper_client_jaas.conf.j2'),
       group = 'hadoop',
+      mode = None,
     )
     self.assertResourceCalled('File', '/etc/zookeeper/conf/zoo_sample.cfg',
       owner = 'zookeeper',
       group = 'hadoop',
     )
-    self.assertNoMoreResources()
\ No newline at end of file
+    self.assertNoMoreResources()

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/test/python/stacks/2.0.6/ZOOKEEPER/test_zookeeper_server.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.0.6/ZOOKEEPER/test_zookeeper_server.py b/ambari-server/src/test/python/stacks/2.0.6/ZOOKEEPER/test_zookeeper_server.py
index 4b9721a..61b3f40 100644
--- a/ambari-server/src/test/python/stacks/2.0.6/ZOOKEEPER/test_zookeeper_server.py
+++ b/ambari-server/src/test/python/stacks/2.0.6/ZOOKEEPER/test_zookeeper_server.py
@@ -21,6 +21,7 @@ from mock.mock import MagicMock, patch
 from stacks.utils.RMFTestCase import *
 
 @patch("os.path.exists", new = MagicMock(return_value=True))
+@patch("platform.linux_distribution", new = MagicMock(return_value="Linux"))
 class TestZookeeperServer(RMFTestCase):
   COMMON_SERVICES_PACKAGE_DIR = "ZOOKEEPER/3.4.5.2.0/package"
   STACK_VERSION = "2.0.6"
@@ -127,11 +128,13 @@ class TestZookeeperServer(RMFTestCase):
       owner = 'zookeeper',
       content = Template('zoo.cfg.j2'),
       group = 'hadoop',
+      mode = None,
     )
     self.assertResourceCalled('File', '/etc/zookeeper/conf/configuration.xsl',
       owner = 'zookeeper',
       content = Template('configuration.xsl.j2'),
       group = 'hadoop',
+      mode = None,
     )
     self.assertResourceCalled('Directory', '/var/run/zookeeper',
       owner = 'zookeeper',
@@ -180,11 +183,13 @@ class TestZookeeperServer(RMFTestCase):
       owner = 'zookeeper',
       content = Template('zoo.cfg.j2'),
       group = 'hadoop',
+      mode = None,
     )
     self.assertResourceCalled('File', '/etc/zookeeper/conf/configuration.xsl',
       owner = 'zookeeper',
       content = Template('configuration.xsl.j2'),
       group = 'hadoop',
+      mode = None,
     )
     self.assertResourceCalled('Directory', '/var/run/zookeeper',
       owner = 'zookeeper',
@@ -217,11 +222,13 @@ class TestZookeeperServer(RMFTestCase):
       owner = 'zookeeper',
       content = Template('zookeeper_jaas.conf.j2'),
       group = 'hadoop',
+      mode = None,
     )
     self.assertResourceCalled('File', '/etc/zookeeper/conf/zookeeper_client_jaas.conf',
       owner = 'zookeeper',
       content = Template('zookeeper_client_jaas.conf.j2'),
       group = 'hadoop',
+      mode = None,
     )
     self.assertResourceCalled('File', '/etc/zookeeper/conf/zoo_sample.cfg',
       owner = 'zookeeper',

http://git-wip-us.apache.org/repos/asf/ambari/blob/e4686db5/ambari-server/src/test/python/stacks/utils/RMFTestCase.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/utils/RMFTestCase.py b/ambari-server/src/test/python/stacks/utils/RMFTestCase.py
index b7e58b4..dc2d861 100644
--- a/ambari-server/src/test/python/stacks/utils/RMFTestCase.py
+++ b/ambari-server/src/test/python/stacks/utils/RMFTestCase.py
@@ -119,6 +119,12 @@ class RMFTestCase(TestCase):
     if 'params' in sys.modules:  
       del(sys.modules["params"])
 
+    if 'params_windows' in sys.modules:
+      del(sys.modules["params_windows"])
+
+    if 'params_linux' in sys.modules:
+      del(sys.modules["params_linux"])
+
     # Reload status_params import, otherwise it won't change properties during next import
     if 'status_params' in sys.modules:
       del(sys.modules["status_params"])


[3/4] ambari git commit: AMBARI-10226: [WinTP2] Merge HDPWIN PIG package scripts to common services (jluniya)

Posted by jl...@apache.org.
AMBARI-10226: [WinTP2] Merge HDPWIN PIG package scripts to common services (jluniya)


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

Branch: refs/heads/trunk
Commit: 626ef03daf847ad4e480ce5eb27fca7f6f9fcb81
Parents: 194df08
Author: Jayush Luniya <jl...@hortonworks.com>
Authored: Fri Mar 27 08:58:57 2015 -0700
Committer: Jayush Luniya <jl...@hortonworks.com>
Committed: Fri Mar 27 08:58:57 2015 -0700

----------------------------------------------------------------------
 .../PIG/0.12.0.2.0/package/scripts/params.py    | 60 +--------------
 .../0.12.0.2.0/package/scripts/params_linux.py  | 78 ++++++++++++++++++++
 .../package/scripts/params_windows.py           | 36 +++++++++
 .../PIG/0.12.0.2.0/package/scripts/pig.py       | 20 ++++-
 .../0.12.0.2.0/package/scripts/pig_client.py    | 26 +++++--
 .../0.12.0.2.0/package/scripts/service_check.py | 16 ++++
 .../2.1/services/PIG/package/scripts/params.py  | 37 ----------
 .../2.1/services/PIG/package/scripts/pig.py     | 49 ------------
 .../services/PIG/package/scripts/pig_client.py  | 41 ----------
 .../PIG/package/scripts/service_check.py        | 34 ---------
 10 files changed, 172 insertions(+), 225 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/626ef03d/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params.py b/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params.py
index 2be49fa..9e3148e 100644
--- a/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params.py
@@ -18,61 +18,9 @@ limitations under the License.
 Ambari Agent
 
 """
+from ambari_commons import OSCheck
 
-from resource_management.libraries.functions.version import format_hdp_stack_version, compare_versions
-from resource_management import *
-
-# server configurations
-config = Script.get_config()
-tmp_dir = Script.get_tmp_dir()
-
-stack_name = default("/hostLevelParams/stack_name", None)
-
-stack_version_unformatted = str(config['hostLevelParams']['stack_version'])
-hdp_stack_version = format_hdp_stack_version(stack_version_unformatted)
-
-# New Cluster Stack Version that is defined during the RESTART of a Rolling Upgrade
-version = default("/commandParams/version", None)
-
-#hadoop params
-if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
-  hadoop_bin_dir = "/usr/hdp/current/hadoop-client/bin"
-  hadoop_home = '/usr/hdp/current/hadoop-client'
-  pig_bin_dir = '/usr/hdp/current/pig-client/bin'
+if OSCheck.is_windows_family():
+  from params_windows import *
 else:
-  hadoop_bin_dir = "/usr/bin"
-  hadoop_home = '/usr'
-  pig_bin_dir = ""
-
-hadoop_conf_dir = "/etc/hadoop/conf"
-pig_conf_dir = "/etc/pig/conf"
-hdfs_user = config['configurations']['hadoop-env']['hdfs_user']
-hdfs_principal_name = config['configurations']['hadoop-env']['hdfs_principal_name']
-hdfs_user_keytab = config['configurations']['hadoop-env']['hdfs_user_keytab']
-smokeuser = config['configurations']['cluster-env']['smokeuser']
-smokeuser_principal = config['configurations']['cluster-env']['smokeuser_principal_name']
-user_group = config['configurations']['cluster-env']['user_group']
-security_enabled = config['configurations']['cluster-env']['security_enabled']
-smoke_user_keytab = config['configurations']['cluster-env']['smokeuser_keytab']
-kinit_path_local = functions.get_kinit_path()
-pig_env_sh_template = config['configurations']['pig-env']['content']
-
-# not supporting 32 bit jdk.
-java64_home = config['hostLevelParams']['java_home']
-
-pig_properties = config['configurations']['pig-properties']['content']
-
-log4j_props = config['configurations']['pig-log4j']['content']
-
-import functools
-#create partial functions with common arguments for every HdfsDirectory call
-#to create hdfs directory we need to call params.HdfsDirectory in code
-HdfsDirectory = functools.partial(
-  HdfsDirectory,
-  conf_dir=hadoop_conf_dir,
-  hdfs_user=hdfs_user,
-  security_enabled = security_enabled,
-  keytab = hdfs_user_keytab,
-  kinit_path_local = kinit_path_local,
-  bin_dir = hadoop_bin_dir
-)
+  from params_linux import *

http://git-wip-us.apache.org/repos/asf/ambari/blob/626ef03d/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params_linux.py
new file mode 100644
index 0000000..2be49fa
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params_linux.py
@@ -0,0 +1,78 @@
+"""
+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.
+
+Ambari Agent
+
+"""
+
+from resource_management.libraries.functions.version import format_hdp_stack_version, compare_versions
+from resource_management import *
+
+# server configurations
+config = Script.get_config()
+tmp_dir = Script.get_tmp_dir()
+
+stack_name = default("/hostLevelParams/stack_name", None)
+
+stack_version_unformatted = str(config['hostLevelParams']['stack_version'])
+hdp_stack_version = format_hdp_stack_version(stack_version_unformatted)
+
+# New Cluster Stack Version that is defined during the RESTART of a Rolling Upgrade
+version = default("/commandParams/version", None)
+
+#hadoop params
+if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
+  hadoop_bin_dir = "/usr/hdp/current/hadoop-client/bin"
+  hadoop_home = '/usr/hdp/current/hadoop-client'
+  pig_bin_dir = '/usr/hdp/current/pig-client/bin'
+else:
+  hadoop_bin_dir = "/usr/bin"
+  hadoop_home = '/usr'
+  pig_bin_dir = ""
+
+hadoop_conf_dir = "/etc/hadoop/conf"
+pig_conf_dir = "/etc/pig/conf"
+hdfs_user = config['configurations']['hadoop-env']['hdfs_user']
+hdfs_principal_name = config['configurations']['hadoop-env']['hdfs_principal_name']
+hdfs_user_keytab = config['configurations']['hadoop-env']['hdfs_user_keytab']
+smokeuser = config['configurations']['cluster-env']['smokeuser']
+smokeuser_principal = config['configurations']['cluster-env']['smokeuser_principal_name']
+user_group = config['configurations']['cluster-env']['user_group']
+security_enabled = config['configurations']['cluster-env']['security_enabled']
+smoke_user_keytab = config['configurations']['cluster-env']['smokeuser_keytab']
+kinit_path_local = functions.get_kinit_path()
+pig_env_sh_template = config['configurations']['pig-env']['content']
+
+# not supporting 32 bit jdk.
+java64_home = config['hostLevelParams']['java_home']
+
+pig_properties = config['configurations']['pig-properties']['content']
+
+log4j_props = config['configurations']['pig-log4j']['content']
+
+import functools
+#create partial functions with common arguments for every HdfsDirectory call
+#to create hdfs directory we need to call params.HdfsDirectory in code
+HdfsDirectory = functools.partial(
+  HdfsDirectory,
+  conf_dir=hadoop_conf_dir,
+  hdfs_user=hdfs_user,
+  security_enabled = security_enabled,
+  keytab = hdfs_user_keytab,
+  kinit_path_local = kinit_path_local,
+  bin_dir = hadoop_bin_dir
+)

http://git-wip-us.apache.org/repos/asf/ambari/blob/626ef03d/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params_windows.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params_windows.py b/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params_windows.py
new file mode 100644
index 0000000..3e63083
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/params_windows.py
@@ -0,0 +1,36 @@
+"""
+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.
+
+Ambari Agent
+
+"""
+from resource_management import *
+
+# server configurations
+config = Script.get_config()
+hdp_root = os.path.abspath(os.path.join(os.environ["HADOOP_HOME"],".."))
+pig_home = os.environ['PIG_HOME']
+pig_conf_dir = os.path.join(pig_home,'conf')
+pig_properties = config['configurations']['pig-properties']['content']
+
+if (('pig-log4j' in config['configurations']) and ('content' in config['configurations']['pig-log4j'])):
+  log4j_props = config['configurations']['pig-log4j']['content']
+else:
+  log4j_props = None
+
+pig_user = "hadoop"
+hdfs_user = "hadoop"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/626ef03d/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/pig.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/pig.py b/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/pig.py
index 036b930..c5f095f 100644
--- a/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/pig.py
+++ b/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/pig.py
@@ -19,9 +19,11 @@ Ambari Agent
 
 """
 import os
-
 from resource_management import *
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
 
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
 def pig():
   import params
 
@@ -58,3 +60,19 @@ def pig():
       group=params.user_group,
       owner=params.hdfs_user
     )
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def pig():
+  import params
+  File(os.path.join(params.pig_conf_dir, "pig.properties"),
+       mode="f",
+       owner=params.pig_user,
+       content=params.pig_properties
+  )
+
+  if (params.log4j_props != None):
+    File(os.path.join(params.pig_conf_dir, "log4j.properties"),
+         mode='f',
+         owner=params.pig_user,
+         content=params.log4j_props
+    )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/626ef03d/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/pig_client.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/pig_client.py b/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/pig_client.py
index eec9304..5b81b1f 100644
--- a/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/pig_client.py
+++ b/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/pig_client.py
@@ -20,12 +20,23 @@ Ambari Agent
 """
 
 import sys
+import os
 from resource_management import *
 from pig import pig
-
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
 
 class PigClient(Script):
+  def configure(self, env):
+    import params
+    env.set_params(params)
+    pig()
 
+  def status(self, env):
+    raise ClientComponentHasNoStatus()
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class PigClientLinux(PigClient):
   def get_stack_to_component(self):
     return {"HDP": "hadoop-client"}
 
@@ -40,13 +51,14 @@ class PigClient(Script):
     self.install_packages(env)
     self.configure(env)
 
-  def configure(self, env):
-    import params
-    env.set_params(params)
-    pig()
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class PigClientWindows(PigClient):
 
-  def status(self, env):
-    raise ClientComponentHasNoStatus()
+  def install(self, env):
+    import params
+    if params.pig_home is None:
+      self.install_packages(env)
+    self.configure(env)
 
 if __name__ == "__main__":
   PigClient().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/626ef03d/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/service_check.py b/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/service_check.py
index 4fbfeb4..9de30ed 100644
--- a/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/service_check.py
+++ b/ambari-server/src/main/resources/common-services/PIG/0.12.0.2.0/package/scripts/service_check.py
@@ -21,8 +21,15 @@ Ambari Agent
 
 from resource_management import *
 from resource_management.libraries.functions.dynamic_variable_interpretation import copy_tarballs_to_hdfs
+from resource_management.libraries import functions
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
 
 class PigServiceCheck(Script):
+  pass
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class PigServiceCheckLinux(PigServiceCheck):
   def service_check(self, env):
     import params
     env.set_params(params)
@@ -104,6 +111,15 @@ class PigServiceCheck(Script):
         bin_dir = params.hadoop_bin_dir
       )
 
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class PigServiceCheckWindows(PigServiceCheck):
+  def service_check(self, env):
+    import params
+    env.set_params(params)
+    smoke_cmd = os.path.join(params.hdp_root,"Run-SmokeTests.cmd")
+    service = "PIG"
+    Execute(format("cmd /C {smoke_cmd} {service}"), logoutput=True, user=params.hdfs_user)
+
 if __name__ == "__main__":
   PigServiceCheck().execute()
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/626ef03d/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/params.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/params.py
deleted file mode 100644
index 854d577..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/params.py
+++ /dev/null
@@ -1,37 +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.
-
-Ambari Agent
-
-"""
-
-from resource_management import *
-
-# server configurations
-config = Script.get_config()
-hdp_root = os.path.abspath(os.path.join(os.environ["HADOOP_HOME"],".."))
-pig_home = os.environ['PIG_HOME']
-pig_conf_dir = os.path.join(pig_home,'conf')
-pig_properties = config['configurations']['pig-properties']['pig-content']
-
-if (('pig-log4j' in config['configurations']) and ('content' in config['configurations']['pig-log4j'])):
-  log4j_props = config['configurations']['pig-log4j']['content']
-else:
-  log4j_props = None
-
-pig_user = "hadoop"
-hdfs_user = "hadoop"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/626ef03d/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/pig.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/pig.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/pig.py
deleted file mode 100644
index 47bb40f..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/pig.py
+++ /dev/null
@@ -1,49 +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.
-
-Ambari Agent
-
-"""
-
-import os
-from resource_management import *
-
-def pig():
-  import params
-  File(os.path.join(params.pig_conf_dir, "pig.properties"),
-       mode="f",
-       owner=params.pig_user,
-       content=params.pig_properties
-  )
-
-  if (params.log4j_props != None):
-    File(os.path.join(params.pig_conf_dir, "log4j.properties"),
-         mode='f',
-         owner=params.pig_user,
-         content=params.log4j_props
-    )
-
-def pig_TemplateConfig(name):
-  import params
-
-  if not isinstance(name, list):
-    name = [name]
-
-  for x in name:
-    TemplateConfig(os.path.join(params.pig_conf_dir,x),
-        owner = params.hdfs_user
-    )

http://git-wip-us.apache.org/repos/asf/ambari/blob/626ef03d/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/pig_client.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/pig_client.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/pig_client.py
deleted file mode 100644
index 381bfe9..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/pig_client.py
+++ /dev/null
@@ -1,41 +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.
-
-Ambari Agent
-
-"""
-
-import os
-from resource_management import *
-
-
-class PigClient(Script):
-  def install(self, env):
-    import params
-    if params.pig_home is None:
-      self.install_packages(env)
-    self.configure(env)
-
-  def configure(self, env):
-    import params
-    env.set_params(params)
-
-  def status(self, env):
-    raise ClientComponentHasNoStatus()
-
-if __name__ == "__main__":
-  PigClient().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/626ef03d/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/service_check.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/service_check.py
deleted file mode 100644
index 9e519c5..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/PIG/package/scripts/service_check.py
+++ /dev/null
@@ -1,34 +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.
-
-Ambari Agent
-
-"""
-
-from resource_management import *
-from resource_management.libraries import functions
-
-class PigServiceCheck(Script):
-  def service_check(self, env):
-    import params
-    env.set_params(params)
-    smoke_cmd = os.path.join(params.hdp_root,"Run-SmokeTests.cmd")
-    service = "PIG"
-    Execute(format("cmd /C {smoke_cmd} {service}"), logoutput=True, user=params.hdfs_user)
-
-if __name__ == "__main__":
-  PigServiceCheck().execute()


[4/4] ambari git commit: AMBARI-10227: [WinTP2] Merge HDPWIN FALCON package scripts to common services (jluniya)

Posted by jl...@apache.org.
AMBARI-10227: [WinTP2] Merge HDPWIN FALCON package scripts to common services (jluniya)


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

Branch: refs/heads/trunk
Commit: 391cc7a6f6bad90888603c31c46c282a3e91a306
Parents: 626ef03
Author: Jayush Luniya <jl...@hortonworks.com>
Authored: Fri Mar 27 09:03:00 2015 -0700
Committer: Jayush Luniya <jl...@hortonworks.com>
Committed: Fri Mar 27 09:03:00 2015 -0700

----------------------------------------------------------------------
 .../FALCON/0.5.0.2.1/package/scripts/falcon.py  |  30 ++++++
 .../0.5.0.2.1/package/scripts/falcon_client.py  |  30 ++++--
 .../0.5.0.2.1/package/scripts/falcon_server.py  |  43 ++++----
 .../FALCON/0.5.0.2.1/package/scripts/params.py  |  91 +---------------
 .../0.5.0.2.1/package/scripts/params_linux.py   | 107 +++++++++++++++++++
 .../0.5.0.2.1/package/scripts/params_windows.py |  46 ++++++++
 .../0.5.0.2.1/package/scripts/service_check.py  |  15 ++-
 .../0.5.0.2.1/package/scripts/status_params.py  |  29 ++---
 .../services/FALCON/package/scripts/falcon.py   |  41 -------
 .../FALCON/package/scripts/falcon_client.py     |  37 -------
 .../FALCON/package/scripts/falcon_server.py     |  44 --------
 .../services/FALCON/package/scripts/params.py   |  45 --------
 .../FALCON/package/scripts/service_check.py     |  33 ------
 .../FALCON/package/scripts/service_mapping.py   |  20 ----
 .../package/templates/client.properties.j2      |  42 --------
 .../stacks/2.1/FALCON/test_falcon_server.py     |   1 +
 16 files changed, 261 insertions(+), 393 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon.py b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon.py
index 7404f04..a56ba5c 100644
--- a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon.py
+++ b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon.py
@@ -19,7 +19,10 @@ limitations under the License.
 
 from resource_management import *
 import os.path
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
 
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
 def falcon(type, action = None):
   import params
   if action == 'config':
@@ -122,3 +125,30 @@ def falcon(type, action = None):
       File(params.server_pid_file,
            action='delete'
       )
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def falcon(type, action = None):
+  import params
+  if action == 'config':
+    env = Environment.get_instance()
+    # These 2 parameters are used in ../templates/client.properties.j2
+    env.config.params["falcon_host"] = params.falcon_host
+    env.config.params["falcon_port"] = params.falcon_port
+    File(os.path.join(params.falcon_conf_dir, 'falcon-env.sh'),
+         content=InlineTemplate(params.falcon_env_sh_template)
+    )
+    File(os.path.join(params.falcon_conf_dir, 'client.properties'),
+         content=Template('client.properties.j2')
+    )
+    PropertiesFile(os.path.join(params.falcon_conf_dir, 'runtime.properties'),
+                   properties=params.falcon_runtime_properties
+    )
+    PropertiesFile(os.path.join(params.falcon_conf_dir, 'startup.properties'),
+                   properties=params.falcon_startup_properties
+    )
+
+  if type == 'server':
+    if action == 'start':
+      Service(params.falcon_win_service_name, action="start")
+    if action == 'stop':
+      Service(params.falcon_win_service_name, action="stop")
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon_client.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon_client.py b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon_client.py
index ea9f154..cf95692 100644
--- a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon_client.py
+++ b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon_client.py
@@ -19,26 +19,27 @@ limitations under the License.
 
 from resource_management import *
 from falcon import falcon
-
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
 
 class FalconClient(Script):
-
-  def get_stack_to_component(self):
-    return {"HDP": "falcon-client"}
-
-  def install(self, env):
-    self.install_packages(env)
-    self.configure(env)
-  
   def configure(self, env):
     import params
-
     env.set_params(params)
     falcon('client', action='config')
 
   def status(self, env):
     raise ClientComponentHasNoStatus()
 
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class FalconClientLinux(FalconClient):
+  def get_stack_to_component(self):
+    return {"HDP": "falcon-client"}
+
+  def install(self, env):
+    self.install_packages(env)
+    self.configure(env)
+
   def pre_rolling_restart(self, env):
     import params
     env.set_params(params)
@@ -60,5 +61,14 @@ class FalconClient(Script):
     else:
       self.put_structured_out({"securityState": "UNSECURED"})
 
+
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class FalconClientWindows(FalconClient):
+  def install(self, env):
+    import params
+    if params.falcon_home is None:
+      self.install_packages(env)
+    self.configure(env)
+
 if __name__ == "__main__":
   FalconClient().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon_server.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon_server.py b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon_server.py
index af98a03..04d14ee 100644
--- a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon_server.py
+++ b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/falcon_server.py
@@ -25,55 +25,44 @@ from resource_management.libraries.functions.security_commons import build_expec
   cached_kinit_executor, get_params_from_filesystem, validate_security_config_properties, \
   FILE_TYPE_PROPERTIES
 from falcon import falcon
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
 
 class FalconServer(Script):
-
-  def get_stack_to_component(self):
-    return {"HDP": "falcon-server"}
-
-  def install(self, env):
+  def configure(self, env):
     import params
-
-    self.install_packages(env)
     env.set_params(params)
-
+    falcon('server', action='config')
 
   def start(self, env, rolling_restart=False):
     import params
-
     env.set_params(params)
     self.configure(env)
-
     falcon('server', action='start')
 
-
   def stop(self, env, rolling_restart=False):
     import params
-
     env.set_params(params)
-
     falcon('server', action='stop')
-
     # if performing an upgrade, backup some directories after stopping falcon
     if rolling_restart:
       falcon_server_upgrade.post_stop_backup()
 
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class FalconServerLinux(FalconServer):
+  def get_stack_to_component(self):
+    return {"HDP": "falcon-server"}
 
-  def configure(self, env):
+  def install(self, env):
     import params
-
+    self.install_packages(env)
     env.set_params(params)
 
-    falcon('server', action='config')
-
-
   def status(self, env):
     import status_params
-
     env.set_params(status_params)
     check_process_status(status_params.server_pid_file)
 
-
   def pre_rolling_restart(self, env):
     import params
     env.set_params(params)
@@ -147,5 +136,17 @@ class FalconServer(Script):
       self.put_structured_out({"securityState": "UNSECURED"})
 
 
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class FalconServerWindows(FalconServer):
+
+  def install(self, env):
+    import params
+    if not check_windows_service_exists(params.falcon_win_service_name):
+      self.install_packages(env)
+
+  def status(self, env):
+    import status_params
+    check_windows_service_status(status_params.falcon_win_service_name)
+
 if __name__ == "__main__":
   FalconServer().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params.py b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params.py
index 49842fa..11a37a2 100644
--- a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params.py
@@ -16,93 +16,10 @@ See the License for the specific language governing permissions and
 limitations under the License.
 
 """
-
-from resource_management.libraries.functions.version import format_hdp_stack_version, compare_versions
-from resource_management.libraries.functions.default import default
-from resource_management import *
-
+from ambari_commons import OSCheck
 from status_params import *
 
-config = Script.get_config()
-
-stack_name = default("/hostLevelParams/stack_name", None)
-
-# New Cluster Stack Version that is defined during the RESTART of a Rolling Upgrade
-version = default("/commandParams/version", None)
-
-stack_version_unformatted = str(config['hostLevelParams']['stack_version'])
-hdp_stack_version = format_hdp_stack_version(stack_version_unformatted)
-
-# hadoop params
-if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
-  hadoop_bin_dir = "/usr/hdp/current/hadoop-client/bin"
-
-  # if this is a server action, then use the server binaries; smoke tests
-  # use the client binaries
-  server_role_dir_mapping = { 'FALCON_SERVER' : 'falcon-server',
-    'FALCON_SERVICE_CHECK' : 'falcon-client' }
-
-  command_role = default("/role", "")
-  if command_role not in server_role_dir_mapping:
-    command_role = 'FALCON_SERVICE_CHECK'
-
-  falcon_root = server_role_dir_mapping[command_role]
-  falcon_webapp_dir = format('/usr/hdp/current/{falcon_root}/webapp')
-  falcon_home = format('/usr/hdp/current/{falcon_root}')
+if OSCheck.is_windows_family():
+  from params_windows import *
 else:
-  hadoop_bin_dir = "/usr/bin"
-  falcon_webapp_dir = '/var/lib/falcon/webapp'
-  falcon_home = '/usr/lib/falcon'
-
-hadoop_conf_dir = "/etc/hadoop/conf"
-falcon_conf_dir_prefix = "/etc/falcon"
-falcon_conf_dir = format("{falcon_conf_dir_prefix}/conf")
-oozie_user = config['configurations']['oozie-env']['oozie_user']
-falcon_user = config['configurations']['falcon-env']['falcon_user']
-smoke_user =  config['configurations']['cluster-env']['smokeuser']
-
-user_group = config['configurations']['cluster-env']['user_group']
-proxyuser_group =  config['configurations']['hadoop-env']['proxyuser_group']
-
-java_home = config['hostLevelParams']['java_home']
-falcon_local_dir = config['configurations']['falcon-env']['falcon_local_dir']
-falcon_log_dir = config['configurations']['falcon-env']['falcon_log_dir']
-
-# falcon-startup.properties
-store_uri = config['configurations']['falcon-startup.properties']['*.config.store.uri']
-# If these properties are present, the directories need to be created.
-falcon_graph_storage_directory = default("/configurations/falcon-startup.properties/*.falcon.graph.storage.directory", None)  # explicitly set in HDP 2.2 and higher
-falcon_graph_serialize_path = default("/configurations/falcon-startup.properties/*.falcon.graph.serialize.path", None)        # explicitly set in HDP 2.2 and higher
-
-falcon_embeddedmq_data = config['configurations']['falcon-env']['falcon.embeddedmq.data']
-falcon_embeddedmq_enabled = config['configurations']['falcon-env']['falcon.embeddedmq']
-falcon_emeddedmq_port = config['configurations']['falcon-env']['falcon.emeddedmq.port']
-
-falcon_host = config['clusterHostInfo']['falcon_server_hosts'][0]
-falcon_port = config['configurations']['falcon-env']['falcon_port']
-falcon_runtime_properties = config['configurations']['falcon-runtime.properties']
-falcon_startup_properties = config['configurations']['falcon-startup.properties']
-smokeuser_keytab = config['configurations']['cluster-env']['smokeuser_keytab']
-falcon_env_sh_template = config['configurations']['falcon-env']['content']
-
-flacon_apps_dir = '/apps/falcon'
-#for create_hdfs_directory
-security_enabled = config['configurations']['cluster-env']['security_enabled']
-hostname = config["hostname"]
-hdfs_user_keytab = config['configurations']['hadoop-env']['hdfs_user_keytab']
-hdfs_user = config['configurations']['hadoop-env']['hdfs_user']
-hdfs_principal_name = config['configurations']['hadoop-env']['hdfs_principal_name']
-smokeuser_principal =  config['configurations']['cluster-env']['smokeuser_principal_name']
-kinit_path_local = functions.get_kinit_path()
-import functools
-#create partial functions with common arguments for every HdfsDirectory call
-#to create hdfs directory we need to call params.HdfsDirectory in code
-HdfsDirectory = functools.partial(
-  HdfsDirectory,
-  conf_dir=hadoop_conf_dir,
-  hdfs_user=hdfs_user,
-  security_enabled = security_enabled,
-  keytab = hdfs_user_keytab,
-  kinit_path_local = kinit_path_local,
-  bin_dir = hadoop_bin_dir
-)
+  from params_linux import *

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params_linux.py
new file mode 100644
index 0000000..e986fdd
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params_linux.py
@@ -0,0 +1,107 @@
+"""
+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.libraries.functions.version import format_hdp_stack_version, compare_versions
+from resource_management.libraries.functions.default import default
+from resource_management import *
+from status_params import *
+
+config = Script.get_config()
+
+stack_name = default("/hostLevelParams/stack_name", None)
+
+# New Cluster Stack Version that is defined during the RESTART of a Rolling Upgrade
+version = default("/commandParams/version", None)
+
+stack_version_unformatted = str(config['hostLevelParams']['stack_version'])
+hdp_stack_version = format_hdp_stack_version(stack_version_unformatted)
+
+# hadoop params
+if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
+  hadoop_bin_dir = "/usr/hdp/current/hadoop-client/bin"
+
+  # if this is a server action, then use the server binaries; smoke tests
+  # use the client binaries
+  server_role_dir_mapping = { 'FALCON_SERVER' : 'falcon-server',
+    'FALCON_SERVICE_CHECK' : 'falcon-client' }
+
+  command_role = default("/role", "")
+  if command_role not in server_role_dir_mapping:
+    command_role = 'FALCON_SERVICE_CHECK'
+
+  falcon_root = server_role_dir_mapping[command_role]
+  falcon_webapp_dir = format('/usr/hdp/current/{falcon_root}/webapp')
+  falcon_home = format('/usr/hdp/current/{falcon_root}')
+else:
+  hadoop_bin_dir = "/usr/bin"
+  falcon_webapp_dir = '/var/lib/falcon/webapp'
+  falcon_home = '/usr/lib/falcon'
+
+hadoop_conf_dir = "/etc/hadoop/conf"
+falcon_conf_dir_prefix = "/etc/falcon"
+falcon_conf_dir = format("{falcon_conf_dir_prefix}/conf")
+oozie_user = config['configurations']['oozie-env']['oozie_user']
+falcon_user = config['configurations']['falcon-env']['falcon_user']
+smoke_user =  config['configurations']['cluster-env']['smokeuser']
+
+user_group = config['configurations']['cluster-env']['user_group']
+proxyuser_group =  config['configurations']['hadoop-env']['proxyuser_group']
+
+java_home = config['hostLevelParams']['java_home']
+falcon_local_dir = config['configurations']['falcon-env']['falcon_local_dir']
+falcon_log_dir = config['configurations']['falcon-env']['falcon_log_dir']
+
+# falcon-startup.properties
+store_uri = config['configurations']['falcon-startup.properties']['*.config.store.uri']
+# If these properties are present, the directories need to be created.
+falcon_graph_storage_directory = default("/configurations/falcon-startup.properties/*.falcon.graph.storage.directory", None)  # explicitly set in HDP 2.2 and higher
+falcon_graph_serialize_path = default("/configurations/falcon-startup.properties/*.falcon.graph.serialize.path", None)        # explicitly set in HDP 2.2 and higher
+
+falcon_embeddedmq_data = config['configurations']['falcon-env']['falcon.embeddedmq.data']
+falcon_embeddedmq_enabled = config['configurations']['falcon-env']['falcon.embeddedmq']
+falcon_emeddedmq_port = config['configurations']['falcon-env']['falcon.emeddedmq.port']
+
+falcon_host = config['clusterHostInfo']['falcon_server_hosts'][0]
+falcon_port = config['configurations']['falcon-env']['falcon_port']
+falcon_runtime_properties = config['configurations']['falcon-runtime.properties']
+falcon_startup_properties = config['configurations']['falcon-startup.properties']
+smokeuser_keytab = config['configurations']['cluster-env']['smokeuser_keytab']
+falcon_env_sh_template = config['configurations']['falcon-env']['content']
+
+flacon_apps_dir = '/apps/falcon'
+#for create_hdfs_directory
+security_enabled = config['configurations']['cluster-env']['security_enabled']
+hostname = config["hostname"]
+hdfs_user_keytab = config['configurations']['hadoop-env']['hdfs_user_keytab']
+hdfs_user = config['configurations']['hadoop-env']['hdfs_user']
+hdfs_principal_name = config['configurations']['hadoop-env']['hdfs_principal_name']
+smokeuser_principal =  config['configurations']['cluster-env']['smokeuser_principal_name']
+kinit_path_local = functions.get_kinit_path()
+import functools
+#create partial functions with common arguments for every HdfsDirectory call
+#to create hdfs directory we need to call params.HdfsDirectory in code
+HdfsDirectory = functools.partial(
+  HdfsDirectory,
+  conf_dir=hadoop_conf_dir,
+  hdfs_user=hdfs_user,
+  security_enabled = security_enabled,
+  keytab = hdfs_user_keytab,
+  kinit_path_local = kinit_path_local,
+  bin_dir = hadoop_bin_dir
+)

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params_windows.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params_windows.py b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params_windows.py
new file mode 100644
index 0000000..a5e23f2
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/params_windows.py
@@ -0,0 +1,46 @@
+"""
+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
+from status_params import *
+
+config = Script.get_config()
+falcon_runtime_properties = config['configurations']['falcon-runtime.properties']
+falcon_startup_properties = config['configurations']['falcon-startup.properties']
+falcon_env_sh_template = config['configurations']['falcon-env']['content']
+
+falcon_host = config['clusterHostInfo']['falcon_server_hosts'][0]
+falcon_port = config['configurations']['falcon-env']['falcon_port']
+
+falcon_conf_dir = "."
+falcon_data_dir = "."
+falcon_home = None
+falcon_log_dir = "."
+
+hdp_root = os.path.abspath(os.path.join(os.environ["HADOOP_HOME"], ".."))
+
+if os.environ.has_key("FALCON_CONF_DIR"):
+  falcon_conf_dir = os.environ["FALCON_CONF_DIR"]
+if os.environ.has_key("FALCON_DATA_DIR"):
+  falcon_data_dir = os.environ["FALCON_DATA_DIR"]
+if os.environ.has_key("FALCON_HOME"):
+  falcon_home = os.environ["FALCON_HOME"]
+if os.environ.has_key("FALCON_LOG_DIR"):
+  falcon_log_dir = os.environ["FALCON_LOG_DIR"]

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/service_check.py b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/service_check.py
index f76718b..678cb02 100644
--- a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/service_check.py
+++ b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/service_check.py
@@ -18,10 +18,14 @@ limitations under the License.
 """
 
 from resource_management import *
-
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
 
 class FalconServiceCheck(Script):
+  pass
 
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class FalconServiceCheckLinux(FalconServiceCheck):
   def service_check(self, env):
     import params
 
@@ -36,5 +40,14 @@ class FalconServiceCheck(Script):
             try_sleep = 20
     )
 
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class FalconServiceCheckWindows(FalconServiceCheck):
+  def service_check(self, env):
+    import params
+    env.set_params(params)
+    smoke_cmd = os.path.join(params.hdp_root,"Run-SmokeTests.cmd")
+    service = "FALCON"
+    Execute(format("cmd /C {smoke_cmd} {service}"), logoutput=True)
+
 if __name__ == "__main__":
   FalconServiceCheck().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/status_params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/status_params.py b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/status_params.py
index 59578f5..bc3d21e 100644
--- a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/status_params.py
+++ b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/package/scripts/status_params.py
@@ -18,18 +18,23 @@ limitations under the License.
 """
 
 from resource_management import *
+from ambari_commons import OSCheck
 
 config = Script.get_config()
-falcon_pid_dir = config['configurations']['falcon-env']['falcon_pid_dir']
-server_pid_file = format('{falcon_pid_dir}/falcon.pid')
 
-# Security related/required params
-hostname = config['hostname']
-security_enabled = config['configurations']['cluster-env']['security_enabled']
-hadoop_conf_dir = "/etc/hadoop/conf"
-kinit_path_local = functions.get_kinit_path()
-tmp_dir = Script.get_tmp_dir()
-falcon_conf_dir_prefix = "/etc/falcon"
-falcon_conf_dir = format("{falcon_conf_dir_prefix}/conf")
-hdfs_user = config['configurations']['hadoop-env']['hdfs_user']
-falcon_user = config['configurations']['falcon-env']['falcon_user']
+if OSCheck.is_windows_family():
+  falcon_win_service_name = "falcon"
+else:
+  falcon_pid_dir = config['configurations']['falcon-env']['falcon_pid_dir']
+  server_pid_file = format('{falcon_pid_dir}/falcon.pid')
+
+  # Security related/required params
+  hostname = config['hostname']
+  security_enabled = config['configurations']['cluster-env']['security_enabled']
+  hadoop_conf_dir = "/etc/hadoop/conf"
+  kinit_path_local = functions.get_kinit_path()
+  tmp_dir = Script.get_tmp_dir()
+  falcon_conf_dir_prefix = "/etc/falcon"
+  falcon_conf_dir = format("{falcon_conf_dir_prefix}/conf")
+  hdfs_user = config['configurations']['hadoop-env']['hdfs_user']
+  falcon_user = config['configurations']['falcon-env']['falcon_user']

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/falcon.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/falcon.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/falcon.py
deleted file mode 100644
index a0b182c..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/falcon.py
+++ /dev/null
@@ -1,41 +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.
-
-"""
-
-from resource_management import *
-
-def falcon():
-  import params
-
-  env = Environment.get_instance()
-  # These 2 parameters are used in ../templates/client.properties.j2
-  env.config.params["falcon_host"] = params.falcon_host
-  env.config.params["falcon_port"] = params.falcon_port
-
-  File(os.path.join(params.falcon_conf_dir, 'falcon-env.sh'),
-       content=InlineTemplate(params.falcon_env_sh_template)
-  )
-  File(os.path.join(params.falcon_conf_dir, 'client.properties'),
-       content=Template('client.properties.j2')
-  )
-  PropertiesFile(os.path.join(params.falcon_conf_dir, 'runtime.properties'),
-                 properties=params.falcon_runtime_properties
-  )
-  PropertiesFile(os.path.join(params.falcon_conf_dir, 'startup.properties'),
-                 properties=params.falcon_startup_properties
-  )

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/falcon_client.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/falcon_client.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/falcon_client.py
deleted file mode 100644
index 86c7545..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/falcon_client.py
+++ /dev/null
@@ -1,37 +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.
-
-"""
-
-from resource_management import *
-
-class FalconClient(Script):
-  def install(self, env):
-    import params
-    if params.falcon_home is None:
-      self.install_packages(env)
-    self.configure(env)
-
-  def configure(self, env):
-    import params
-    env.set_params(params)
-
-  def status(self, env):
-    raise ClientComponentHasNoStatus()
-
-if __name__ == "__main__":
-  FalconClient().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/falcon_server.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/falcon_server.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/falcon_server.py
deleted file mode 100644
index 225456b..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/falcon_server.py
+++ /dev/null
@@ -1,44 +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.
-
-"""
-
-from resource_management import *
-from falcon import falcon
-import service_mapping
-
-class FalconServer(Script):
-  def install(self, env):
-    if not check_windows_service_exists(service_mapping.falcon_win_service_name):
-      self.install_packages(env)
-
-  def start(self, env):
-    import params
-    self.configure(env)
-    Service(service_mapping.falcon_win_service_name, action="start")
-
-  def stop(self, env):
-    Service(service_mapping.falcon_win_service_name, action="stop")
-
-  def configure(self, env):
-    falcon()
-
-  def status(self, env):
-    check_windows_service_status(service_mapping.falcon_win_service_name)
-
-if __name__ == "__main__":
-  FalconServer().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/params.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/params.py
deleted file mode 100644
index 815f45b..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/params.py
+++ /dev/null
@@ -1,45 +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.
-
-"""
-
-from resource_management import *
-import os
-
-config = Script.get_config()
-falcon_runtime_properties = config['configurations']['falcon-runtime.properties']
-falcon_startup_properties = config['configurations']['falcon-startup.properties']
-falcon_env_sh_template = config['configurations']['falcon-env']['content']
-
-falcon_host = config['clusterHostInfo']['falcon_server_hosts'][0]
-falcon_port = config['configurations']['falcon-env']['falcon_port']
-
-falcon_conf_dir = "."
-falcon_data_dir = "."
-falcon_home = None
-falcon_log_dir = "."
-
-hdp_root = os.path.abspath(os.path.join(os.environ["HADOOP_HOME"], ".."))
-
-if os.environ.has_key("FALCON_CONF_DIR"):
-  falcon_conf_dir = os.environ["FALCON_CONF_DIR"]
-if os.environ.has_key("FALCON_DATA_DIR"):
-  falcon_data_dir = os.environ["FALCON_DATA_DIR"]
-if os.environ.has_key("FALCON_HOME"):
-  falcon_home = os.environ["FALCON_HOME"]
-if os.environ.has_key("FALCON_LOG_DIR"):
-  falcon_log_dir = os.environ["FALCON_LOG_DIR"]

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/service_check.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/service_check.py
deleted file mode 100644
index 3dcfb5b..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/service_check.py
+++ /dev/null
@@ -1,33 +0,0 @@
-"""
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-
-"""
-
-from resource_management import *
-
-
-class FalconServiceCheck(Script):
-
-  def service_check(self, env):
-    import params
-    env.set_params(params)
-    smoke_cmd = os.path.join(params.hdp_root,"Run-SmokeTests.cmd")
-    service = "FALCON"
-    Execute(format("cmd /C {smoke_cmd} {service}"), logoutput=True)
-
-if __name__ == "__main__":
-  FalconServiceCheck().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/service_mapping.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/service_mapping.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/service_mapping.py
deleted file mode 100644
index fc4d9f9..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/scripts/service_mapping.py
+++ /dev/null
@@ -1,20 +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.
-
-"""
-
-falcon_win_service_name = "falcon"

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/templates/client.properties.j2
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/templates/client.properties.j2 b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/templates/client.properties.j2
deleted file mode 100644
index 63749db..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/FALCON/package/templates/client.properties.j2
+++ /dev/null
@@ -1,42 +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.
-#}
-
-#
-# 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.
-#
-
-#########################################################################
-##########    This is used for falcon packaging only. ###################
-## Uses default port. Please change if configured for non-default port ##
-#########################################################################
-
-falcon.url=http://{{falcon_host}}:{{falcon_port}}/

http://git-wip-us.apache.org/repos/asf/ambari/blob/391cc7a6/ambari-server/src/test/python/stacks/2.1/FALCON/test_falcon_server.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.1/FALCON/test_falcon_server.py b/ambari-server/src/test/python/stacks/2.1/FALCON/test_falcon_server.py
index 7d52b4b..5bfa749 100644
--- a/ambari-server/src/test/python/stacks/2.1/FALCON/test_falcon_server.py
+++ b/ambari-server/src/test/python/stacks/2.1/FALCON/test_falcon_server.py
@@ -21,6 +21,7 @@ limitations under the License.
 from mock.mock import MagicMock, patch
 from stacks.utils.RMFTestCase import *
 
+@patch("platform.linux_distribution", new = MagicMock(return_value="Linux"))
 class TestFalconServer(RMFTestCase):
   COMMON_SERVICES_PACKAGE_DIR = "FALCON/0.5.0.2.1/package"
   STACK_VERSION = "2.1"


[2/4] ambari git commit: AMBARI-10225: [WinTP2] Merge HDPWIN TEZ package scripts to common services (jluniya)

Posted by jl...@apache.org.
AMBARI-10225: [WinTP2] Merge HDPWIN TEZ package scripts to common services (jluniya)


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

Branch: refs/heads/trunk
Commit: 194df087131fe9a254e6c0e2c47499305bc72075
Parents: e4686db
Author: Jayush Luniya <jl...@hortonworks.com>
Authored: Fri Mar 27 08:54:00 2015 -0700
Committer: Jayush Luniya <jl...@hortonworks.com>
Committed: Fri Mar 27 08:54:00 2015 -0700

----------------------------------------------------------------------
 .../TEZ/0.4.0.2.1/package/scripts/params.py     | 58 ++-------------
 .../0.4.0.2.1/package/scripts/params_linux.py   | 76 ++++++++++++++++++++
 .../0.4.0.2.1/package/scripts/params_windows.py | 30 ++++++++
 .../0.4.0.2.1/package/scripts/service_check.py  | 14 ++++
 .../TEZ/0.4.0.2.1/package/scripts/tez.py        | 21 +++---
 .../TEZ/0.4.0.2.1/package/scripts/tez_client.py | 26 +++++--
 .../2.1/services/TEZ/package/scripts/params.py  | 30 --------
 .../2.1/services/TEZ/package/scripts/tez.py     | 31 --------
 .../services/TEZ/package/scripts/tez_client.py  | 42 -----------
 9 files changed, 155 insertions(+), 173 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/194df087/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params.py b/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params.py
index b58bcd6..a10c1d4 100644
--- a/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params.py
@@ -17,59 +17,9 @@ See the License for the specific language governing permissions and
 limitations under the License.
 
 """
+from ambari_commons import OSCheck
 
-from resource_management.libraries.functions.version import format_hdp_stack_version, compare_versions
-from resource_management import *
-
-# server configurations
-config = Script.get_config()
-tmp_dir = Script.get_tmp_dir()
-
-stack_name = default("/hostLevelParams/stack_name", None)
-
-# This is expected to be of the form #.#.#.#
-stack_version_unformatted = str(config['hostLevelParams']['stack_version'])
-hdp_stack_version = format_hdp_stack_version(stack_version_unformatted)
-
-# New Cluster Stack Version that is defined during the RESTART of a Rolling Upgrade
-version = default("/commandParams/version", None)
-
-if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
-  hadoop_bin_dir = "/usr/hdp/current/hadoop-client/bin"
-  path_to_tez_examples_jar = "/usr/hdp/{hdp_version}/tez/tez-examples*.jar"
+if OSCheck.is_windows_family():
+  from params_windows import *
 else:
-  hadoop_bin_dir = "/usr/bin"
-  path_to_tez_examples_jar = "/usr/lib/tez/tez-mapreduce-examples*.jar"
-hadoop_conf_dir = "/etc/hadoop/conf"
-
-kinit_path_local = functions.get_kinit_path()
-security_enabled = config['configurations']['cluster-env']['security_enabled']
-smokeuser = config['configurations']['cluster-env']['smokeuser']
-smokeuser_principal = config['configurations']['cluster-env']['smokeuser_principal_name']
-smoke_user_keytab = config['configurations']['cluster-env']['smokeuser_keytab']
-hdfs_user = config['configurations']['hadoop-env']['hdfs_user']
-hdfs_principal_name = config['configurations']['hadoop-env']['hdfs_principal_name']
-hdfs_user_keytab = config['configurations']['hadoop-env']['hdfs_user_keytab']
-
-config_dir_prefix = "/etc/tez"
-config_dir = format("{config_dir_prefix}/conf")
-
-hadoop_home = '/usr'
-java64_home = config['hostLevelParams']['java_home']
-
-tez_user = config['configurations']['tez-env']['tez_user']
-user_group = config['configurations']['cluster-env']['user_group']
-tez_env_sh_template = config['configurations']['tez-env']['content']
-
-import functools
-# Create partial functions with common arguments for every HdfsDirectory call
-# to create hdfs directory we need to call params.HdfsDirectory in code
-HdfsDirectory = functools.partial(
-  HdfsDirectory,
-  conf_dir=hadoop_conf_dir,
-  hdfs_user=hdfs_user,
-  security_enabled=security_enabled,
-  keytab=hdfs_user_keytab,
-  kinit_path_local=kinit_path_local,
-  bin_dir=hadoop_bin_dir
-)
+  from params_linux import *

http://git-wip-us.apache.org/repos/asf/ambari/blob/194df087/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params_linux.py
new file mode 100644
index 0000000..3245ff0
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params_linux.py
@@ -0,0 +1,76 @@
+#!/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.libraries.functions.version import format_hdp_stack_version, compare_versions
+from resource_management import *
+
+# server configurations
+config = Script.get_config()
+tmp_dir = Script.get_tmp_dir()
+
+stack_name = default("/hostLevelParams/stack_name", None)
+
+# This is expected to be of the form #.#.#.#
+stack_version_unformatted = str(config['hostLevelParams']['stack_version'])
+hdp_stack_version = format_hdp_stack_version(stack_version_unformatted)
+
+# New Cluster Stack Version that is defined during the RESTART of a Rolling Upgrade
+version = default("/commandParams/version", None)
+
+if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
+  hadoop_bin_dir = "/usr/hdp/current/hadoop-client/bin"
+  path_to_tez_examples_jar = "/usr/hdp/{hdp_version}/tez/tez-examples*.jar"
+else:
+  hadoop_bin_dir = "/usr/bin"
+  path_to_tez_examples_jar = "/usr/lib/tez/tez-mapreduce-examples*.jar"
+hadoop_conf_dir = "/etc/hadoop/conf"
+
+kinit_path_local = functions.get_kinit_path()
+security_enabled = config['configurations']['cluster-env']['security_enabled']
+smokeuser = config['configurations']['cluster-env']['smokeuser']
+smokeuser_principal = config['configurations']['cluster-env']['smokeuser_principal_name']
+smoke_user_keytab = config['configurations']['cluster-env']['smokeuser_keytab']
+hdfs_user = config['configurations']['hadoop-env']['hdfs_user']
+hdfs_principal_name = config['configurations']['hadoop-env']['hdfs_principal_name']
+hdfs_user_keytab = config['configurations']['hadoop-env']['hdfs_user_keytab']
+
+config_dir_prefix = "/etc/tez"
+config_dir = format("{config_dir_prefix}/conf")
+
+hadoop_home = '/usr'
+java64_home = config['hostLevelParams']['java_home']
+
+tez_user = config['configurations']['tez-env']['tez_user']
+user_group = config['configurations']['cluster-env']['user_group']
+tez_env_sh_template = config['configurations']['tez-env']['content']
+
+import functools
+# Create partial functions with common arguments for every HdfsDirectory call
+# to create hdfs directory we need to call params.HdfsDirectory in code
+HdfsDirectory = functools.partial(
+  HdfsDirectory,
+  conf_dir=hadoop_conf_dir,
+  hdfs_user=hdfs_user,
+  security_enabled=security_enabled,
+  keytab=hdfs_user_keytab,
+  kinit_path_local=kinit_path_local,
+  bin_dir=hadoop_bin_dir
+)
+

http://git-wip-us.apache.org/repos/asf/ambari/blob/194df087/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params_windows.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params_windows.py b/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params_windows.py
new file mode 100644
index 0000000..db34c9f
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/params_windows.py
@@ -0,0 +1,30 @@
+"""
+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
+
+config = Script.get_config()
+tez_user = "hadoop"
+tez_home_dir = None
+tez_conf_dir = "conf"
+
+if os.environ.has_key("TEZ_HOME"):
+  tez_home_dir = os.environ["TEZ_HOME"]
+  tez_conf_dir = os.path.join(tez_home_dir, "conf")

http://git-wip-us.apache.org/repos/asf/ambari/blob/194df087/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/service_check.py b/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/service_check.py
index 434b505..35ae5b6 100644
--- a/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/service_check.py
+++ b/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/service_check.py
@@ -22,8 +22,14 @@ Ambari Agent
 from resource_management import *
 from resource_management.libraries.functions.version import compare_versions
 from resource_management.libraries.functions.dynamic_variable_interpretation import copy_tarballs_to_hdfs
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyImpl
 
 class TezServiceCheck(Script):
+  pass
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class TezServiceCheckLinux(TezServiceCheck):
   def service_check(self, env):
     import params
     env.set_params(params)
@@ -91,6 +97,14 @@ class TezServiceCheck(Script):
     )
 
 
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class TezServiceCheckWindows(TezServiceCheck):
+  def service_check(self, env):
+    import params
+    env.set_params(params)
+    smoke_cmd = os.path.join(params.hdp_root,"Run-SmokeTests.cmd")
+    service = "TEZ"
+    Execute(format("cmd /C {smoke_cmd} {service}"), logoutput=True, user=params.tez_user)
 
 
 if __name__ == "__main__":

http://git-wip-us.apache.org/repos/asf/ambari/blob/194df087/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/tez.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/tez.py b/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/tez.py
index bb8b7c2..ceafa56 100644
--- a/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/tez.py
+++ b/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/tez.py
@@ -20,7 +20,10 @@ Ambari Agent
 """
 
 from resource_management import *
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
 
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
 def tez():
   import params
 
@@ -48,14 +51,14 @@ def tez():
   )
 
 
-def tez_TemplateConfig(name):
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def tez():
   import params
-
-  if not isinstance(name, list):
-    name = [name]
-
-  for x in name:
-    TemplateConfig(format("{config_dir}/{x}"),
-                   owner = params.tez_user
-    )
+  XmlConfig("tez-site.xml",
+             conf_dir=params.tez_conf_dir,
+             configurations=params.config['configurations']['tez-site'],
+             owner=params.tez_user,
+             mode="f",
+             configuration_attributes=params.config['configuration_attributes']['tez-site']
+  )
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/194df087/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/tez_client.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/tez_client.py b/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/tez_client.py
index 00375d7..9746d13 100644
--- a/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/tez_client.py
+++ b/ambari-server/src/main/resources/common-services/TEZ/0.4.0.2.1/package/scripts/tez_client.py
@@ -22,8 +22,20 @@ Ambari Agent
 import sys
 from resource_management import *
 from tez import tez
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyImpl
 
 class TezClient(Script):
+  def configure(self, env):
+    import params
+    env.set_params(params)
+    tez()
+
+  def status(self, env):
+    raise ClientComponentHasNoStatus()
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class TezClientLinux(TezClient):
 
   def get_stack_to_component(self):
     return {"HDP": "hadoop-client"}
@@ -39,14 +51,14 @@ class TezClient(Script):
     self.install_packages(env)
     self.configure(env)
 
-  def configure(self, env):
-    import params
-    env.set_params(params)
-    tez()
-
-  def status(self, env):
-    raise ClientComponentHasNoStatus()
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class TezClientWindows(TezClient):
 
+  def install(self, env):
+    import params
+    if params.tez_home_dir is None:
+      self.install_packages(env)
+    self.configure(env)
 
 if __name__ == "__main__":
   TezClient().execute()

http://git-wip-us.apache.org/repos/asf/ambari/blob/194df087/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/TEZ/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/TEZ/package/scripts/params.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/TEZ/package/scripts/params.py
deleted file mode 100644
index db34c9f..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/TEZ/package/scripts/params.py
+++ /dev/null
@@ -1,30 +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.
-
-"""
-
-from resource_management import *
-import os
-
-config = Script.get_config()
-tez_user = "hadoop"
-tez_home_dir = None
-tez_conf_dir = "conf"
-
-if os.environ.has_key("TEZ_HOME"):
-  tez_home_dir = os.environ["TEZ_HOME"]
-  tez_conf_dir = os.path.join(tez_home_dir, "conf")

http://git-wip-us.apache.org/repos/asf/ambari/blob/194df087/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/TEZ/package/scripts/tez.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/TEZ/package/scripts/tez.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/TEZ/package/scripts/tez.py
deleted file mode 100644
index d9f5a38..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/TEZ/package/scripts/tez.py
+++ /dev/null
@@ -1,31 +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.
-
-"""
-
-from resource_management import *
-import sys
-
-def tez():
-    import params
-    XmlConfig("tez-site.xml",
-              conf_dir=params.tez_conf_dir,
-              configurations=params.config['configurations']['tez-site'],
-              owner=params.tez_user,
-              mode="f",
-              configuration_attributes=params.config['configuration_attributes']['tez-site']
-    )

http://git-wip-us.apache.org/repos/asf/ambari/blob/194df087/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/TEZ/package/scripts/tez_client.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/TEZ/package/scripts/tez_client.py b/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/TEZ/package/scripts/tez_client.py
deleted file mode 100644
index 89dd6bb..0000000
--- a/ambari-server/src/main/resources/stacks/HDPWIN/2.1/services/TEZ/package/scripts/tez_client.py
+++ /dev/null
@@ -1,42 +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.
-
-Ambari Agent
-
-"""
-
-import sys
-from resource_management import *
-from tez import tez
-
-class TezClient(Script):
-  def install(self, env):
-    import params
-    if params.tez_home_dir is None:
-      self.install_packages(env)
-    self.configure(env)
-
-  def configure(self, env):
-    import params
-    env.set_params(params)
-    tez()
-
-  def status(self, env):
-    raise ClientComponentHasNoStatus()
-
-if __name__ == "__main__":
-  TezClient().execute()