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:10 UTC

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

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"