You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bigtop.apache.org by rv...@apache.org on 2017/03/22 06:10:04 UTC

[13/52] bigtop git commit: ODPI-193. Add Hive 1.2 to ODPi ambari reference implementation

http://git-wip-us.apache.org/repos/asf/bigtop/blob/ba8d7f50/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server_upgrade.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server_upgrade.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server_upgrade.py
new file mode 100755
index 0000000..478c240
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_server_upgrade.py
@@ -0,0 +1,141 @@
+"""
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+"""
+
+import os
+import re
+from resource_management.core.logger import Logger
+from resource_management.core.exceptions import Fail
+from resource_management.core.resources.system import Execute
+from resource_management.core import shell
+from resource_management.libraries.functions import format
+from resource_management.libraries.functions import stack_select
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+from resource_management.libraries.functions.version import format_stack_version
+
+
+def post_upgrade_deregister():
+  """
+  Runs the "hive --service hiveserver2 --deregister <version>" command to
+  de-provision the server in preparation for an upgrade. This will contact
+  ZooKeeper to remove the server so that clients that attempt to connect
+  will be directed to other servers automatically. Once all
+  clients have drained, the server will shutdown automatically; this process
+  could take a very long time.
+  This function will obtain the Kerberos ticket if security is enabled.
+  :return:
+  """
+  import params
+
+  Logger.info('HiveServer2 executing "deregister" command to complete upgrade...')
+
+  if params.security_enabled:
+    kinit_command=format("{kinit_path_local} -kt {smoke_user_keytab} {smokeuser_principal}; ")
+    Execute(kinit_command,user=params.smokeuser)
+
+  # calculate the current hive server version
+  current_hiveserver_version = _get_current_hiveserver_version()
+  if current_hiveserver_version is None:
+    raise Fail('Unable to determine the current HiveServer2 version to deregister.')
+
+  # fallback when upgrading because <stack-root>/current/hive-server2/conf/conf.server may not exist
+  hive_server_conf_dir = params.hive_server_conf_dir
+  if not os.path.exists(hive_server_conf_dir):
+    hive_server_conf_dir = "/etc/hive/conf.server"
+
+  # deregister
+  hive_execute_path = params.execute_path
+  # If upgrading, the upgrade-target hive binary should be used to call the --deregister command.
+  # If downgrading, the downgrade-source hive binary should be used to call the --deregister command.
+  # By now <stack-selector-tool> has been called to set 'current' to target-stack
+  if "downgrade" == params.upgrade_direction:
+    # hive_bin
+    downgrade_version = params.current_version
+    if params.downgrade_from_version:
+      downgrade_version = params.downgrade_from_version
+    hive_execute_path = _get_hive_execute_path(downgrade_version)
+
+  command = format('hive --config {hive_server_conf_dir} --service hiveserver2 --deregister ' + current_hiveserver_version)
+  Execute(command, user=params.hive_user, path=hive_execute_path, tries=1 )
+
+
+def _get_hive_execute_path(stack_version_formatted):
+  """
+  Returns the exact execute path to use for the given stack-version.
+  This method does not return the "current" path
+  :param stack_version_formatted: Exact stack-version to use in the new path
+  :return: Hive execute path for the exact stack-version
+  """
+  import params
+
+  hive_execute_path = params.execute_path
+  formatted_stack_version = format_stack_version(stack_version_formatted)
+  if formatted_stack_version and check_stack_feature(StackFeature.ROLLING_UPGRADE, formatted_stack_version):
+    # hive_bin
+    new_hive_bin = format('{stack_root}/{stack_version_formatted}/hive/bin')
+    if (os.pathsep + params.hive_bin) in hive_execute_path:
+      hive_execute_path = hive_execute_path.replace(os.pathsep + params.hive_bin, os.pathsep + new_hive_bin)
+    # hadoop_bin_dir
+    new_hadoop_bin = stack_select.get_hadoop_dir_for_stack_version("bin", stack_version_formatted)
+    old_hadoop_bin = params.hadoop_bin_dir
+    if new_hadoop_bin and len(new_hadoop_bin) > 0 and (os.pathsep + old_hadoop_bin) in hive_execute_path:
+      hive_execute_path = hive_execute_path.replace(os.pathsep + old_hadoop_bin, os.pathsep + new_hadoop_bin)
+  return hive_execute_path
+
+
+def _get_current_hiveserver_version():
+  """
+  Runs "hive --version" and parses the result in order
+  to obtain the current version of hive.
+
+  :return:  the hiveserver2 version, returned by "hive --version"
+  """
+  import params
+
+  try:
+    # When downgrading the source version should be the version we are downgrading from
+    if "downgrade" == params.upgrade_direction:
+      if not params.downgrade_from_version:
+        raise Fail('The version from which we are downgrading from should be provided in \'downgrade_from_version\'')
+      source_version = params.downgrade_from_version
+    else:
+      source_version = params.current_version
+    hive_execute_path = _get_hive_execute_path(source_version)
+    version_hive_bin = params.hive_bin
+    formatted_source_version = format_stack_version(source_version)
+    if formatted_source_version and check_stack_feature(StackFeature.ROLLING_UPGRADE, formatted_source_version):
+      version_hive_bin = format('{stack_root}/{source_version}/hive/bin')
+    command = format('{version_hive_bin}/hive --version')
+    return_code, output = shell.call(command, user=params.hive_user, path=hive_execute_path)
+  except Exception, e:
+    Logger.error(str(e))
+    raise Fail('Unable to execute hive --version command to retrieve the hiveserver2 version.')
+
+  if return_code != 0:
+    raise Fail('Unable to determine the current HiveServer2 version because of a non-zero return code of {0}'.format(str(return_code)))
+
+  match = re.search('^(Hive) ([0-9]+.[0-9]+.\S+)', output, re.MULTILINE)
+
+  if match:
+    current_hive_server_version = match.group(2)
+    return current_hive_server_version
+  else:
+    raise Fail('The extracted hiveserver2 version "{0}" does not matching any known pattern'.format(output))
+
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/ba8d7f50/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service.py
new file mode 100755
index 0000000..22b4061
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service.py
@@ -0,0 +1,187 @@
+#!/usr/bin/env python
+"""
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+"""
+
+import os
+import time
+
+from ambari_commons.constants import UPGRADE_TYPE_ROLLING
+from resource_management.core.logger import Logger
+from resource_management.core import shell
+from resource_management.libraries.functions.format import format
+from resource_management.core.resources.system import File, Execute
+from resource_management.core.resources.service import Service
+from resource_management.core.exceptions import Fail
+from resource_management.core.shell import as_user
+from resource_management.libraries.functions.hive_check import check_thrift_port_sasl
+from resource_management.libraries.functions import get_user_call_output
+from resource_management.libraries.functions.show_logs import show_logs
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from ambari_commons import OSConst
+
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def hive_service(name, action='start', upgrade_type=None):
+  import params
+  if name == 'metastore':
+    if action == 'start' or action == 'stop':
+      Service(params.hive_metastore_win_service_name, action=action)
+
+  if name == 'hiveserver2':
+    if action == 'start' or action == 'stop':
+      Service(params.hive_server_win_service_name, action=action)
+
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def hive_service(name, action='start', upgrade_type=None):
+
+  import params
+
+  if name == 'metastore':
+    pid_file = format("{hive_pid_dir}/{hive_metastore_pid}")
+    cmd = format("{start_metastore_path} {hive_log_dir}/hive.out {hive_log_dir}/hive.err {pid_file} {hive_server_conf_dir} {hive_log_dir}")
+  elif name == 'hiveserver2':
+    pid_file = format("{hive_pid_dir}/{hive_pid}")
+    cmd = format("{start_hiveserver2_path} {hive_log_dir}/hive-server2.out {hive_log_dir}/hive-server2.err {pid_file} {hive_server_conf_dir} {hive_log_dir}")
+
+
+    if params.security_enabled and params.current_version and check_stack_feature(StackFeature.HIVE_SERVER2_KERBERIZED_ENV, params.current_version):
+      hive_kinit_cmd = format("{kinit_path_local} -kt {hive_server2_keytab} {hive_principal}; ")
+      Execute(hive_kinit_cmd, user=params.hive_user)
+
+  pid = get_user_call_output.get_user_call_output(format("cat {pid_file}"), user=params.hive_user, is_checked_call=False)[1]
+  process_id_exists_command = format("ls {pid_file} >/dev/null 2>&1 && ps -p {pid} >/dev/null 2>&1")
+
+  if action == 'start':
+    if name == 'hiveserver2':
+      check_fs_root(params.hive_server_conf_dir, params.execute_path)
+
+    daemon_cmd = cmd
+    hadoop_home = params.hadoop_home
+    hive_bin = "hive"
+
+    # upgrading hiveserver2 (rolling_restart) means that there is an existing,
+    # de-registering hiveserver2; the pid will still exist, but the new
+    # hiveserver is spinning up on a new port, so the pid will be re-written
+    if upgrade_type == UPGRADE_TYPE_ROLLING:
+      process_id_exists_command = None
+
+      if params.version and params.stack_root:
+        hadoop_home = format("{stack_root}/{version}/hadoop")
+        hive_bin = os.path.join(params.hive_bin, hive_bin)
+      
+    Execute(daemon_cmd, 
+      user = params.hive_user,
+      environment = { 'HADOOP_HOME': hadoop_home, 'JAVA_HOME': params.java64_home, 'HIVE_BIN': hive_bin },
+      path = params.execute_path,
+      not_if = process_id_exists_command)
+
+    if params.hive_jdbc_driver == "com.mysql.jdbc.Driver" or \
+       params.hive_jdbc_driver == "org.postgresql.Driver" or \
+       params.hive_jdbc_driver == "oracle.jdbc.driver.OracleDriver":
+
+      validation_called = False
+
+      if params.hive_jdbc_target is not None:
+        validation_called = True
+        validate_connection(params.hive_jdbc_target, params.hive_lib)
+      if params.hive2_jdbc_target is not None:
+        validation_called = True
+        validate_connection(params.hive2_jdbc_target, params.hive_server2_hive2_lib)
+
+      if not validation_called:
+        emessage = "ERROR! DB connection check should be executed at least one time!"
+        Logger.error(emessage)
+
+  elif action == 'stop':
+
+    daemon_kill_cmd = format("{sudo} kill {pid}")
+    daemon_hard_kill_cmd = format("{sudo} kill -9 {pid}")
+
+    Execute(daemon_kill_cmd,
+      not_if = format("! ({process_id_exists_command})")
+    )
+
+    wait_time = 5
+    Execute(daemon_hard_kill_cmd,
+      not_if = format("! ({process_id_exists_command}) || ( sleep {wait_time} && ! ({process_id_exists_command}) )"),
+      ignore_failures = True
+    )
+
+    try:
+      # check if stopped the process, else fail the task
+      Execute(format("! ({process_id_exists_command})"),
+        tries=20,
+        try_sleep=3,
+      )
+    except:
+      show_logs(params.hive_log_dir, params.hive_user)
+      raise
+
+    File(pid_file,
+         action = "delete"
+    )
+
+def validate_connection(target_path_to_jdbc, hive_lib_path):
+  import params
+
+  path_to_jdbc = target_path_to_jdbc
+  if not params.jdbc_jar_name:
+    path_to_jdbc = format("{hive_lib_path}/") + \
+                   params.default_connectors_map[params.hive_jdbc_driver] if params.hive_jdbc_driver in params.default_connectors_map else None
+    if not os.path.isfile(path_to_jdbc):
+      path_to_jdbc = format("{hive_lib_path}/") + "*"
+      error_message = "Error! Sorry, but we can't find jdbc driver with default name " + params.default_connectors_map[params.hive_jdbc_driver] + \
+                      " in hive lib dir. So, db connection check can fail. Please run 'ambari-server setup --jdbc-db={db_name} --jdbc-driver={path_to_jdbc} on server host.'"
+      Logger.error(error_message)
+
+  db_connection_check_command = format(
+    "{java64_home}/bin/java -cp {check_db_connection_jar}:{path_to_jdbc} org.apache.ambari.server.DBConnectionVerification '{hive_jdbc_connection_url}' {hive_metastore_user_name} {hive_metastore_user_passwd!p} {hive_jdbc_driver}")
+
+  try:
+    Execute(db_connection_check_command,
+            path='/usr/sbin:/sbin:/usr/local/bin:/bin:/usr/bin', tries=5, try_sleep=10)
+  except:
+    show_logs(params.hive_log_dir, params.hive_user)
+    raise
+
+
+def check_fs_root(conf_dir, execution_path):
+  import params
+
+  if not params.fs_root.startswith("hdfs://"):
+    Logger.info("Skipping fs root check as fs_root does not start with hdfs://")
+    return
+
+  metatool_cmd = format("hive --config {conf_dir} --service metatool")
+  cmd = as_user(format("{metatool_cmd} -listFSRoot", env={'PATH': execution_path}), params.hive_user) \
+        + format(" 2>/dev/null | grep hdfs:// | cut -f1,2,3 -d '/' | grep -v '{fs_root}' | head -1")
+  code, out = shell.call(cmd)
+
+  if code == 0 and out.strip() != "" and params.fs_root.strip() != out.strip():
+    out = out.strip()
+    cmd = format("{metatool_cmd} -updateLocation {fs_root} {out}")
+    Execute(cmd,
+            user=params.hive_user,
+            environment={'PATH': execution_path}
+    )
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/ba8d7f50/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service_interactive.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service_interactive.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service_interactive.py
new file mode 100755
index 0000000..eaf95ad
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/hive_service_interactive.py
@@ -0,0 +1,109 @@
+#!/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.
+
+"""
+
+# Python Imports
+
+# Ambari Commons & Resource Management imports
+import os
+from resource_management.core.logger import Logger
+from resource_management.libraries.functions.format import format
+from resource_management.core.resources.system import File, Execute
+from resource_management.libraries.functions import get_user_call_output
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from ambari_commons import OSConst
+
+# Local Imports
+from hive_service import check_fs_root
+
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def hive_service_interactive(name, action='start', upgrade_type=None):
+  pass
+
+
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def hive_service_interactive(name, action='start', upgrade_type=None):
+  import params
+
+  pid_file = format("{hive_pid_dir}/{hive_interactive_pid}")
+  cmd = format("{start_hiveserver2_interactive_path} {hive_pid_dir}/hive-server2-interactive.out {hive_log_dir}/hive-server2-interactive.err {pid_file} {hive_server_interactive_conf_dir} {hive_log_dir}")
+
+  pid = get_user_call_output.get_user_call_output(format("cat {pid_file}"), user=params.hive_user, is_checked_call=False)[1]
+  process_id_exists_command = format("ls {pid_file} >/dev/null 2>&1 && ps -p {pid} >/dev/null 2>&1")
+
+  if action == 'start':
+    check_fs_root(params.hive_server_interactive_conf_dir, params.execute_path_hive_interactive)
+    daemon_cmd = cmd
+    hadoop_home = params.hadoop_home
+    hive_interactive_bin = "hive2"
+
+    Execute(daemon_cmd,
+            user = params.hive_user,
+            environment = { 'HADOOP_HOME': hadoop_home, 'JAVA_HOME': params.java64_home, 'HIVE_BIN': hive_interactive_bin },
+            path = params.execute_path,
+            not_if = process_id_exists_command)
+
+    if params.hive_jdbc_driver == "com.mysql.jdbc.Driver" or \
+        params.hive_jdbc_driver == "org.postgresql.Driver" or \
+        params.hive_jdbc_driver == "oracle.jdbc.driver.OracleDriver":
+
+      path_to_jdbc = params.target_hive_interactive
+      if not params.jdbc_jar_name:
+        path_to_jdbc = format("{hive_interactive_lib}/") + \
+                       params.default_connectors_map[params.hive_jdbc_driver] if params.hive_jdbc_driver in params.default_connectors_map else None
+        if not os.path.isfile(path_to_jdbc):
+          path_to_jdbc = format("{hive_interactive_lib}/") + "*"
+          error_message = "Error! Sorry, but we can't find jdbc driver with default name " + params.default_connectors_map[params.hive_jdbc_driver] + \
+                " in hive lib dir. So, db connection check can fail. Please run 'ambari-server setup --jdbc-db={db_name} --jdbc-driver={path_to_jdbc} on server host.'"
+          Logger.error(error_message)
+
+      db_connection_check_command = format(
+        "{java64_home}/bin/java -cp {check_db_connection_jar}:{path_to_jdbc} org.apache.ambari.server.DBConnectionVerification '{hive_jdbc_connection_url}' {hive_metastore_user_name} {hive_metastore_user_passwd!p} {hive_jdbc_driver}")
+      Execute(db_connection_check_command,
+              path='/usr/sbin:/sbin:/usr/local/bin:/bin:/usr/bin', tries=5, try_sleep=10)
+  elif action == 'stop':
+
+    daemon_kill_cmd = format("{sudo} kill {pid}")
+    daemon_hard_kill_cmd = format("{sudo} kill -9 {pid}")
+
+    Execute(daemon_kill_cmd,
+            not_if = format("! ({process_id_exists_command})")
+            )
+
+    # check if stopped the process, otherwise send hard kill command.
+    try:
+      Execute(format("! ({process_id_exists_command})"),
+              tries=10,
+              try_sleep=3,
+              )
+    except:
+      Execute(daemon_hard_kill_cmd,
+              not_if = format("! ({process_id_exists_command}) ")
+              )
+
+    # check if stopped the process, else fail the task
+    Execute(format("! ({process_id_exists_command})"),
+            tries=20,
+            try_sleep=3,
+            )
+
+    File(pid_file,
+         action = "delete"
+         )

http://git-wip-us.apache.org/repos/asf/bigtop/blob/ba8d7f50/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_server.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_server.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_server.py
new file mode 100755
index 0000000..851dc02
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_server.py
@@ -0,0 +1,64 @@
+#!/usr/bin/env python
+"""
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+"""
+
+import sys
+import os
+import mysql_users
+from resource_management import *
+
+from mysql_service import mysql_service
+from mysql_utils import mysql_configure
+
+
+class MysqlServer(Script):
+  def install(self, env):
+    import params
+    self.install_packages(env)
+    self.configure(env)
+
+  def clean(self, env):
+    import params
+    env.set_params(params)
+    mysql_users.mysql_deluser()
+
+  def configure(self, env):
+    import params
+    env.set_params(params)
+    mysql_configure()
+
+  def start(self, env, rolling_restart=False):
+    import params
+    env.set_params(params)
+    mysql_service(daemon_name=params.daemon_name, action='start')
+
+  def stop(self, env, rolling_restart=False):
+    import params
+    env.set_params(params)
+    mysql_service(daemon_name=params.daemon_name, action='stop')
+
+  def status(self, env):
+    import status_params
+    env.set_params(status_params)
+
+    mysql_service(daemon_name=status_params.daemon_name, action='status')
+
+
+if __name__ == "__main__":
+  MysqlServer().execute()

http://git-wip-us.apache.org/repos/asf/bigtop/blob/ba8d7f50/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_service.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_service.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_service.py
new file mode 100755
index 0000000..8b98ed1
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_service.py
@@ -0,0 +1,49 @@
+#!/usr/bin/env python
+"""
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+"""
+
+from resource_management import *
+
+
+def mysql_service(daemon_name=None, action='start'): 
+  status_cmd = format("pgrep -l '^{process_name}$'")
+  cmd = ('service', daemon_name, action)
+
+  if action == 'status':
+    try:
+      Execute(status_cmd)
+    except Fail:
+      raise ComponentIsNotRunning()
+  elif action == 'stop':
+    import params
+    Execute(cmd,
+            logoutput = True,
+            only_if = status_cmd,
+            sudo = True,
+    )
+  elif action == 'start':
+    import params   
+    Execute(cmd,
+      logoutput = True,
+      not_if = status_cmd,
+      sudo = True,
+    )
+
+
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/ba8d7f50/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_users.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_users.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_users.py
new file mode 100755
index 0000000..c023548
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_users.py
@@ -0,0 +1,70 @@
+#!/usr/bin/env python
+"""
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+"""
+
+from resource_management import *
+
+# Used to add hive access to the needed components
+def mysql_adduser():
+  import params
+  
+  File(params.mysql_adduser_path,
+       mode=0755,
+       content=StaticFile('addMysqlUser.sh')
+  )
+  hive_server_host = format("{hive_server_host}")
+  hive_metastore_host = format("{hive_metastore_host}")
+
+  add_metastore_cmd = "bash -x {mysql_adduser_path} {daemon_name} {hive_metastore_user_name} {hive_metastore_user_passwd!p} {hive_metastore_host}"
+  add_hiveserver_cmd = "bash -x {mysql_adduser_path} {daemon_name} {hive_metastore_user_name} {hive_metastore_user_passwd!p} {hive_server_host}"
+  if (hive_server_host == hive_metastore_host):
+    cmd = format(add_hiveserver_cmd)
+  else:
+    cmd = format(add_hiveserver_cmd + ";" + add_metastore_cmd)
+  Execute(cmd,
+          tries=3,
+          try_sleep=5,
+          logoutput=False,
+          path='/usr/sbin:/sbin:/usr/local/bin:/bin:/usr/bin'
+  )
+
+# Removes hive access from components
+def mysql_deluser():
+  import params
+  
+  File(params.mysql_deluser_path,
+       mode=0755,
+       content=StaticFile('removeMysqlUser.sh')
+  )
+  hive_server_host = format("{hive_server_host}")
+  hive_metastore_host = format("{hive_metastore_host}")
+
+  del_hiveserver_cmd = "bash -x {mysql_deluser_path} {daemon_name} {hive_metastore_user_name} {hive_server_host}"
+  del_metastore_cmd = "bash -x {mysql_deluser_path} {daemon_name} {hive_metastore_user_name} {hive_metastore_host}"
+  if (hive_server_host == hive_metastore_host):
+    cmd = format(del_hiveserver_cmd)
+  else:
+    cmd = format(
+      del_hiveserver_cmd + ";" + del_metastore_cmd)
+  Execute(cmd,
+          tries=3,
+          try_sleep=5,
+          path='/usr/sbin:/sbin:/usr/local/bin:/bin:/usr/bin',
+  )
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/ba8d7f50/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_utils.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_utils.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_utils.py
new file mode 100755
index 0000000..5006b56
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/mysql_utils.py
@@ -0,0 +1,35 @@
+#!/usr/bin/env python
+"""
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+"""
+
+from resource_management import *
+import mysql_users
+
+def mysql_configure():
+  import params
+
+  # required for running hive
+  replace_bind_address = ('sed','-i','s|^bind-address[ \t]*=.*|bind-address = 0.0.0.0|',params.mysql_configname)
+  Execute(replace_bind_address,
+          sudo = True,
+  )
+  
+  # this also will start mysql-server
+  mysql_users.mysql_adduser()
+  
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/bigtop/blob/ba8d7f50/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params.py
new file mode 100755
index 0000000..f10a3f3
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params.py
@@ -0,0 +1,29 @@
+#!/usr/bin/env python
+"""
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+"""
+from ambari_commons import OSCheck
+from resource_management.libraries.functions.default import default
+
+if OSCheck.is_windows_family():
+  from params_windows import *
+else:
+  from params_linux import *
+
+host_sys_prepped = default("/hostLevelParams/host_sys_prepped", False)
+retryAble = default("/commandParams/command_retry_enabled", False)

http://git-wip-us.apache.org/repos/asf/bigtop/blob/ba8d7f50/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py
new file mode 100755
index 0000000..1cd58c3
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_linux.py
@@ -0,0 +1,736 @@
+#!/usr/bin/env python
+"""
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+"""
+
+import status_params
+import ambari_simplejson as json # simplejson is much faster comparing to Python 2.6 json module and has the same functions set.
+import os
+
+from urlparse import urlparse
+
+from ambari_commons.constants import AMBARI_SUDO_BINARY
+from ambari_commons.os_check import OSCheck
+
+from resource_management.libraries.resources.hdfs_resource import HdfsResource
+from resource_management.libraries.functions.default import default
+from resource_management.libraries.functions.format import format
+from resource_management.libraries.functions.is_empty import is_empty
+from resource_management.libraries.functions.copy_tarball import STACK_ROOT_PATTERN, STACK_NAME_PATTERN, STACK_VERSION_PATTERN
+from resource_management.libraries.functions import get_kinit_path
+from resource_management.libraries.functions.get_not_managed_resources import get_not_managed_resources
+from resource_management.libraries.script.script import Script
+from resource_management.libraries.functions import StackFeature
+from resource_management.libraries.functions.stack_features import check_stack_feature
+from resource_management.libraries.functions.stack_features import get_stack_feature_version
+from resource_management.libraries.functions.get_port_from_url import get_port_from_url
+from resource_management.libraries.functions.expect import expect
+from resource_management.libraries import functions
+from resource_management.libraries.functions.setup_atlas_hook import has_atlas_in_cluster
+
+# Default log4j version; put config files under /etc/hive/conf
+log4j_version = '1'
+
+# server configurations
+config = Script.get_config()
+tmp_dir = Script.get_tmp_dir()
+sudo = AMBARI_SUDO_BINARY
+
+stack_root = status_params.stack_root
+stack_name = status_params.stack_name
+stack_name_uppercase = stack_name.upper()
+agent_stack_retry_on_unavailability = config['hostLevelParams']['agent_stack_retry_on_unavailability']
+agent_stack_retry_count = expect("/hostLevelParams/agent_stack_retry_count", int)
+
+# Needed since this is an Atlas Hook service.
+cluster_name = config['clusterName']
+
+# node hostname
+hostname = config["hostname"]
+
+# This is expected to be of the form #.#.#.#
+stack_version_unformatted = status_params.stack_version_unformatted
+stack_version_formatted_major = status_params.stack_version_formatted_major
+
+# this is not available on INSTALL action because <stack-selector-tool> is not available
+stack_version_formatted = functions.get_stack_version('hive-server2')
+
+# New Cluster Stack Version that is defined during the RESTART of a Rolling Upgrade.
+# It cannot be used during the initial Cluser Install because the version is not yet known.
+version = default("/commandParams/version", None)
+
+# current host stack version
+current_version = default("/hostLevelParams/current_version", None)
+
+# When downgrading the 'version' and 'current_version' are both pointing to the downgrade-target version
+# downgrade_from_version provides the source-version the downgrade is happening from
+downgrade_from_version = default("/commandParams/downgrade_from_version", None)
+
+# get the correct version to use for checking stack features
+version_for_stack_feature_checks = get_stack_feature_version(config)
+
+# Upgrade direction
+upgrade_direction = default("/commandParams/upgrade_direction", None)
+stack_supports_ranger_kerberos = check_stack_feature(StackFeature.RANGER_KERBEROS_SUPPORT, version_for_stack_feature_checks)
+stack_supports_ranger_audit_db = check_stack_feature(StackFeature.RANGER_AUDIT_DB_SUPPORT, version_for_stack_feature_checks)
+
+# component ROLE directory (like hive-metastore or hive-server2-hive2)
+component_directory = status_params.component_directory
+component_directory_interactive = status_params.component_directory_interactive
+
+hadoop_home = format('{stack_root}/current/hadoop-client')
+hive_bin = format('{stack_root}/current/{component_directory}/bin')
+hive_schematool_ver_bin = format('{stack_root}/{version}/hive/bin')
+hive_schematool_bin = format('{stack_root}/current/{component_directory}/bin')
+hive_lib = format('{stack_root}/current/{component_directory}/lib')
+hive_version_lib = format('{stack_root}/{version}/hive/lib')
+hive_var_lib = '/var/lib/hive'
+hive_user_home_dir = "/home/hive"
+
+# starting on stacks where HSI is supported, we need to begin using the 'hive2' schematool
+hive_server2_hive2_dir = None
+hive_server2_hive2_lib = None
+
+version = default("/commandParams/version", None)
+
+if check_stack_feature(StackFeature.HIVE_SERVER_INTERACTIVE, version_for_stack_feature_checks):
+  # the name of the hiveserver2-hive2 component
+  hive_server2_hive2_component = status_params.SERVER_ROLE_DIRECTORY_MAP["HIVE_SERVER_INTERACTIVE"]
+
+  # when using the version, we can just specify the component as "hive2"
+  hive_schematool_ver_bin = format('{stack_root}/{version}/hive2/bin')
+
+  # use the schematool which ships with hive2
+  hive_schematool_bin = format('{stack_root}/current/{hive_server2_hive2_component}/bin')
+
+  # <stack-root>/<version>/hive2 (as opposed to <stack-root>/<version>/hive)
+  hive_server2_hive2_dir = format('{stack_root}/current/{hive_server2_hive2_component}')
+
+  # <stack-root>/<version>/hive2 (as opposed to <stack-root>/<version>/hive)
+  hive_server2_hive2_version_dir = format('{stack_root}/{version}/hive2')
+
+  # <stack-root>/current/hive-server2-hive2/lib -> <stack-root>/<version>/hive2/lib
+  hive_server2_hive2_lib = format('{hive_server2_hive2_dir}/lib')
+
+  # <stack-root>/<version>/hive2/lib
+  hive_server2_hive2_version_lib = format('{hive_server2_hive2_version_dir}/lib')
+
+
+hive_interactive_bin = format('{stack_root}/current/{component_directory_interactive}/bin')
+hive_interactive_lib = format('{stack_root}/current/{component_directory_interactive}/lib')
+
+# Hive Interactive related paths
+hive_interactive_var_lib = '/var/lib/hive2'
+
+# These tar folders were used in previous stack versions, e.g., HDP 2.1
+hadoop_streaming_jars = '/usr/lib/hadoop-mapreduce/hadoop-streaming-*.jar'
+pig_tar_file = format('/usr/share/{stack_name_uppercase}-webhcat/pig.tar.gz')
+hive_tar_file = format('/usr/share/{stack_name_uppercase}-webhcat/hive.tar.gz')
+sqoop_tar_file = format('/usr/share/{stack_name_uppercase}-webhcat/sqoop*.tar.gz')
+
+hive_metastore_site_supported = False
+hive_etc_dir_prefix = "/etc/hive"
+hive_interactive_etc_dir_prefix = "/etc/hive2"
+limits_conf_dir = "/etc/security/limits.d"
+
+hive_user_nofile_limit = default("/configurations/hive-env/hive_user_nofile_limit", "32000")
+hive_user_nproc_limit = default("/configurations/hive-env/hive_user_nproc_limit", "16000")
+
+# use the directories from status_params as they are already calculated for
+# the correct stack version
+hadoop_conf_dir = status_params.hadoop_conf_dir
+hadoop_bin_dir = status_params.hadoop_bin_dir
+webhcat_conf_dir = status_params.webhcat_conf_dir
+hive_conf_dir = status_params.hive_conf_dir
+hive_home_dir = status_params.hive_home_dir
+hive_config_dir = status_params.hive_config_dir
+hive_client_conf_dir = status_params.hive_client_conf_dir
+hive_server_conf_dir = status_params.hive_server_conf_dir
+
+hcat_conf_dir = '/etc/hive-hcatalog/conf'
+config_dir = '/etc/hive-webhcat/conf'
+hcat_lib = '/usr/lib/hive-hcatalog/share/hcatalog'
+webhcat_bin_dir = '/usr/lib/hive-hcatalog/sbin'
+
+# there are no client versions of these, use server versions directly
+hcat_lib = format('{stack_root}/current/hive-webhcat/share/hcatalog')
+webhcat_bin_dir = format('{stack_root}/current/hive-webhcat/sbin')
+
+# --- Tarballs ---
+# DON'T CHANGE THESE VARIABLE NAMES
+# Values don't change from those in copy_tarball.py
+webhcat_apps_dir = "/apps/webhcat"
+hive_tar_source = "{0}/{1}/hive/hive.tar.gz".format(STACK_ROOT_PATTERN, STACK_VERSION_PATTERN)
+pig_tar_source = "{0}/{1}/pig/pig.tar.gz".format(STACK_ROOT_PATTERN, STACK_VERSION_PATTERN)
+hive_tar_dest_file = "/{0}/apps/{1}/hive/hive.tar.gz".format(STACK_NAME_PATTERN,STACK_VERSION_PATTERN)
+pig_tar_dest_file = "/{0}/apps/{1}/pig/pig.tar.gz".format(STACK_NAME_PATTERN, STACK_VERSION_PATTERN)
+
+hadoop_streaming_tar_source = "{0}/{1}/hadoop-mapreduce/hadoop-streaming.jar".format(STACK_ROOT_PATTERN, STACK_VERSION_PATTERN)
+sqoop_tar_source = "{0}/{1}/sqoop/sqoop.tar.gz".format(STACK_ROOT_PATTERN, STACK_VERSION_PATTERN)
+hadoop_streaming_tar_dest_dir = "/{0}/apps/{1}/mapreduce/".format(STACK_NAME_PATTERN,STACK_VERSION_PATTERN)
+sqoop_tar_dest_dir = "/{0}/apps/{1}/sqoop/".format(STACK_NAME_PATTERN, STACK_VERSION_PATTERN)
+
+tarballs_mode = 0444
+
+purge_tables = "false"
+# Starting from stack version for feature hive_purge_table drop should be executed with purge
+if check_stack_feature(StackFeature.HIVE_PURGE_TABLE, version_for_stack_feature_checks):
+  purge_tables = 'true'
+
+if check_stack_feature(StackFeature.HIVE_WEBHCAT_SPECIFIC_CONFIGS, version_for_stack_feature_checks):
+  # this is NOT a typo.  Configs for hcatalog/webhcat point to a
+  # specific directory which is NOT called 'conf'
+  hcat_conf_dir = format('{stack_root}/current/hive-webhcat/etc/hcatalog')
+  config_dir = format('{stack_root}/current/hive-webhcat/etc/webhcat')
+
+if check_stack_feature(StackFeature.HIVE_METASTORE_SITE_SUPPORT, version_for_stack_feature_checks):
+  hive_metastore_site_supported = True
+
+execute_path = os.environ['PATH'] + os.pathsep + hive_bin + os.pathsep + hadoop_bin_dir
+
+hive_metastore_user_name = config['configurations']['hive-site']['javax.jdo.option.ConnectionUserName']
+hive_jdbc_connection_url = config['configurations']['hive-site']['javax.jdo.option.ConnectionURL']
+
+hive_metastore_user_passwd = config['configurations']['hive-site']['javax.jdo.option.ConnectionPassword']
+hive_metastore_user_passwd = unicode(hive_metastore_user_passwd) if not is_empty(hive_metastore_user_passwd) else hive_metastore_user_passwd
+hive_metastore_db_type = config['configurations']['hive-env']['hive_database_type']
+
+#HACK Temporarily use dbType=azuredb while invoking schematool
+if hive_metastore_db_type == "mssql":
+  hive_metastore_db_type = "azuredb"
+
+#users
+hive_user = config['configurations']['hive-env']['hive_user']
+
+#JDBC driver jar name
+hive_jdbc_driver = config['configurations']['hive-site']['javax.jdo.option.ConnectionDriverName']
+jdk_location = config['hostLevelParams']['jdk_location']
+java_share_dir = '/usr/share/java'
+hive_database_name = config['configurations']['hive-env']['hive_database_name']
+hive_database = config['configurations']['hive-env']['hive_database']
+hive_use_existing_db = hive_database.startswith('Existing')
+
+default_connectors_map = { "com.microsoft.sqlserver.jdbc.SQLServerDriver":"sqljdbc4.jar",
+                           "com.mysql.jdbc.Driver":"mysql-connector-java.jar",
+                           "org.postgresql.Driver":"postgresql-jdbc.jar",
+                           "oracle.jdbc.driver.OracleDriver":"ojdbc.jar",
+                           "sap.jdbc4.sqlanywhere.IDriver":"sajdbc4.jar"}
+
+# NOT SURE THAT IT'S A GOOD IDEA TO USE PATH TO CLASS IN DRIVER, MAYBE IT WILL BE BETTER TO USE DB TYPE.
+# BECAUSE PATH TO CLASSES COULD BE CHANGED
+sqla_db_used = False
+hive_previous_jdbc_jar_name = None
+if hive_jdbc_driver == "com.microsoft.sqlserver.jdbc.SQLServerDriver":
+  jdbc_jar_name = default("/hostLevelParams/custom_mssql_jdbc_name", None)
+  hive_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_mssql_jdbc_name", None)
+elif hive_jdbc_driver == "com.mysql.jdbc.Driver":
+  jdbc_jar_name = default("/hostLevelParams/custom_mysql_jdbc_name", None)
+  hive_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_mysql_jdbc_name", None)
+elif hive_jdbc_driver == "org.postgresql.Driver":
+  jdbc_jar_name = default("/hostLevelParams/custom_postgres_jdbc_name", None)
+  hive_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_postgres_jdbc_name", None)
+elif hive_jdbc_driver == "oracle.jdbc.driver.OracleDriver":
+  jdbc_jar_name = default("/hostLevelParams/custom_oracle_jdbc_name", None)
+  hive_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_oracle_jdbc_name", None)
+elif hive_jdbc_driver == "sap.jdbc4.sqlanywhere.IDriver":
+  jdbc_jar_name = default("/hostLevelParams/custom_sqlanywhere_jdbc_name", None)
+  hive_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_sqlanywhere_jdbc_name", None)
+  sqla_db_used = True
+
+default_mysql_jar_name = "mysql-connector-java.jar"
+default_mysql_target = format("{hive_lib}/{default_mysql_jar_name}")
+hive_previous_jdbc_jar = format("{hive_lib}/{hive_previous_jdbc_jar_name}")
+if not hive_use_existing_db:
+  jdbc_jar_name = default_mysql_jar_name
+
+
+downloaded_custom_connector = format("{tmp_dir}/{jdbc_jar_name}")
+
+hive_jdbc_target = format("{hive_lib}/{jdbc_jar_name}")
+hive2_jdbc_target = None
+if hive_server2_hive2_dir:
+  hive2_jdbc_target = format("{hive_server2_hive2_lib}/{jdbc_jar_name}")
+
+# during upgrade / downgrade, use the specific version to copy the JDBC JAR to
+if upgrade_direction:
+  hive_jdbc_target = format("{hive_version_lib}/{jdbc_jar_name}")
+  hive2_jdbc_target = format("{hive_server2_hive2_version_lib}/{jdbc_jar_name}") if hive2_jdbc_target is not None else None
+
+
+hive2_previous_jdbc_jar = format("{hive_server2_hive2_lib}/{hive_previous_jdbc_jar_name}") if hive_server2_hive2_lib is not None else None
+driver_curl_source = format("{jdk_location}/{jdbc_jar_name}")
+
+# normally, the JDBC driver would be referenced by <stack-root>/current/.../foo.jar
+# but in RU if <stack-selector-tool> is called and the restart fails, then this means that current pointer
+# is now pointing to the upgraded version location; that's bad for the cp command
+source_jdbc_file = format("{stack_root}/{current_version}/hive/lib/{jdbc_jar_name}")
+
+check_db_connection_jar_name = "DBConnectionVerification.jar"
+check_db_connection_jar = format("/usr/lib/ambari-agent/{check_db_connection_jar_name}")
+hive_jdbc_drivers_list = ["com.microsoft.sqlserver.jdbc.SQLServerDriver","com.mysql.jdbc.Driver",
+                          "org.postgresql.Driver","oracle.jdbc.driver.OracleDriver","sap.jdbc4.sqlanywhere.IDriver"]
+
+prepackaged_jdbc_name = "ojdbc6.jar"
+prepackaged_ojdbc_symlink = format("{hive_lib}/{prepackaged_jdbc_name}")
+templeton_port = config['configurations']['webhcat-site']['templeton.port']
+
+#constants for type2 jdbc
+jdbc_libs_dir = format("{hive_lib}/native/lib64")
+lib_dir_available = os.path.exists(jdbc_libs_dir)
+
+if sqla_db_used:
+  jars_path_in_archive = format("{tmp_dir}/sqla-client-jdbc/java/*")
+  libs_path_in_archive = format("{tmp_dir}/sqla-client-jdbc/native/lib64/*")
+  downloaded_custom_connector = format("{tmp_dir}/{jdbc_jar_name}")
+  libs_in_hive_lib = format("{jdbc_libs_dir}/*")
+
+
+# Start, Common Hosts and Ports
+ambari_server_hostname = config['clusterHostInfo']['ambari_server_host'][0]
+
+hive_metastore_hosts = default('/clusterHostInfo/hive_metastore_host', [])
+hive_metastore_host = hive_metastore_hosts[0] if len(hive_metastore_hosts) > 0 else None
+hive_metastore_port = get_port_from_url(config['configurations']['hive-site']['hive.metastore.uris'])
+
+hive_server_hosts = default("/clusterHostInfo/hive_server_host", [])
+hive_server_host = hive_server_hosts[0] if len(hive_server_hosts) > 0 else None
+
+hive_server_interactive_hosts = default('/clusterHostInfo/hive_server_interactive_hosts', [])
+hive_server_interactive_host = hive_server_interactive_hosts[0] if len(hive_server_interactive_hosts) > 0 else None
+# End, Common Hosts and Ports
+
+hive_transport_mode = config['configurations']['hive-site']['hive.server2.transport.mode']
+
+if hive_transport_mode.lower() == "http":
+  hive_server_port = config['configurations']['hive-site']['hive.server2.thrift.http.port']
+else:
+  hive_server_port = default('/configurations/hive-site/hive.server2.thrift.port',"10000")
+
+hive_url = format("jdbc:hive2://{hive_server_host}:{hive_server_port}")
+hive_http_endpoint = default('/configurations/hive-site/hive.server2.thrift.http.path', "cliservice")
+hive_server_principal = config['configurations']['hive-site']['hive.server2.authentication.kerberos.principal']
+hive_server2_authentication = config['configurations']['hive-site']['hive.server2.authentication']
+
+# ssl options
+hive_ssl = default('/configurations/hive-site/hive.server2.use.SSL', False)
+hive_ssl_keystore_path = default('/configurations/hive-site/hive.server2.keystore.path', None)
+hive_ssl_keystore_password = default('/configurations/hive-site/hive.server2.keystore.password', None)
+
+smokeuser = config['configurations']['cluster-env']['smokeuser']
+smoke_test_sql = format("{tmp_dir}/hiveserver2.sql")
+smoke_test_path = format("{tmp_dir}/hiveserver2Smoke.sh")
+smoke_user_keytab = config['configurations']['cluster-env']['smokeuser_keytab']
+smokeuser_principal = config['configurations']['cluster-env']['smokeuser_principal_name']
+
+fs_root = config['configurations']['core-site']['fs.defaultFS']
+security_enabled = config['configurations']['cluster-env']['security_enabled']
+
+kinit_path_local = get_kinit_path(default('/configurations/kerberos-env/executable_search_paths', None))
+hive_metastore_keytab_path = config['configurations']['hive-site']['hive.metastore.kerberos.keytab.file']
+hive_metastore_principal = config['configurations']['hive-site']['hive.metastore.kerberos.principal']
+
+hive_server2_keytab = config['configurations']['hive-site']['hive.server2.authentication.kerberos.keytab']
+
+#hive_env
+hive_log_dir = config['configurations']['hive-env']['hive_log_dir']
+hive_pid_dir = status_params.hive_pid_dir
+hive_pid = status_params.hive_pid
+hive_interactive_pid = status_params.hive_interactive_pid
+
+#Default conf dir for client
+hive_conf_dirs_list = [hive_client_conf_dir]
+
+# These are the folders to which the configs will be written to.
+ranger_hive_component = status_params.SERVER_ROLE_DIRECTORY_MAP['HIVE_SERVER']
+if status_params.role == "HIVE_METASTORE" and hive_metastore_hosts is not None and hostname in hive_metastore_hosts:
+  hive_conf_dirs_list.append(hive_server_conf_dir)
+elif status_params.role == "HIVE_SERVER" and hive_server_hosts is not None and hostname in hive_server_host:
+  hive_conf_dirs_list.append(hive_server_conf_dir)
+elif status_params.role == "HIVE_SERVER_INTERACTIVE" and hive_server_interactive_hosts is not None and hostname in hive_server_interactive_hosts:
+  hive_conf_dirs_list.append(status_params.hive_server_interactive_conf_dir)
+  ranger_hive_component = status_params.SERVER_ROLE_DIRECTORY_MAP['HIVE_SERVER_INTERACTIVE']
+# log4j version is 2 for hive2; put config files under /etc/hive2/conf
+if status_params.role == "HIVE_SERVER_INTERACTIVE":
+  log4j_version = '2'
+
+#Starting hiveserver2
+start_hiveserver2_script = 'startHiveserver2.sh.j2'
+
+##Starting metastore
+start_metastore_script = 'startMetastore.sh'
+hive_metastore_pid = status_params.hive_metastore_pid
+
+# Hive Server Interactive
+slider_am_container_mb = default("/configurations/hive-interactive-env/slider_am_container_mb", 341)
+
+hdfs_user = config['configurations']['hadoop-env']['hdfs_user']
+yarn_user = config['configurations']['yarn-env']['yarn_user']
+user_group = config['configurations']['cluster-env']['user_group']
+artifact_dir = format("{tmp_dir}/AMBARI-artifacts/")
+# Need this for yarn.nodemanager.recovery.dir in yarn-site
+yarn_log_dir_prefix = config['configurations']['yarn-env']['yarn_log_dir_prefix']
+
+target_hive_interactive = format("{hive_interactive_lib}/{jdbc_jar_name}")
+hive_intaractive_previous_jdbc_jar = format("{hive_interactive_lib}/{hive_previous_jdbc_jar_name}")
+jars_in_hive_lib = format("{hive_lib}/*.jar")
+
+start_hiveserver2_path = format("{tmp_dir}/start_hiveserver2_script")
+start_metastore_path = format("{tmp_dir}/start_metastore_script")
+
+hadoop_heapsize = config['configurations']['hadoop-env']['hadoop_heapsize']
+
+if 'role' in config and config['role'] in ["HIVE_SERVER", "HIVE_METASTORE"]:
+  if check_stack_feature(StackFeature.HIVE_ENV_HEAPSIZE, version_for_stack_feature_checks):
+    hive_heapsize = config['configurations']['hive-env']['hive.heapsize']
+  else:
+    hive_heapsize = config['configurations']['hive-site']['hive.heapsize']
+else:
+  hive_heapsize = config['configurations']['hive-env']['hive.client.heapsize']
+
+hive_metastore_heapsize = config['configurations']['hive-env']['hive.metastore.heapsize']
+
+java64_home = config['hostLevelParams']['java_home']
+java_version = expect("/hostLevelParams/java_version", int)
+
+##### MYSQL
+db_name = config['configurations']['hive-env']['hive_database_name']
+mysql_group = 'mysql'
+mysql_host = config['clusterHostInfo']['hive_mysql_host']
+
+mysql_adduser_path = format("{tmp_dir}/addMysqlUser.sh")
+mysql_deluser_path = format("{tmp_dir}/removeMysqlUser.sh")
+
+#### Metastore
+# initialize the schema only if not in an upgrade/downgrade
+init_metastore_schema = upgrade_direction is None
+
+########## HCAT
+hcat_dbroot = hcat_lib
+
+hcat_user = config['configurations']['hive-env']['hcat_user']
+webhcat_user = config['configurations']['hive-env']['webhcat_user']
+
+hcat_pid_dir = status_params.hcat_pid_dir
+hcat_log_dir = config['configurations']['hive-env']['hcat_log_dir']
+hcat_env_sh_template = config['configurations']['hcat-env']['content']
+
+#hive-log4j.properties.template
+if (('hive-log4j' in config['configurations']) and ('content' in config['configurations']['hive-log4j'])):
+  log4j_props = config['configurations']['hive-log4j']['content']
+else:
+  log4j_props = None
+
+#webhcat-log4j.properties.template
+if (('webhcat-log4j' in config['configurations']) and ('content' in config['configurations']['webhcat-log4j'])):
+  log4j_webhcat_props = config['configurations']['webhcat-log4j']['content']
+else:
+  log4j_webhcat_props = None
+
+#hive-exec-log4j.properties.template
+if (('hive-exec-log4j' in config['configurations']) and ('content' in config['configurations']['hive-exec-log4j'])):
+  log4j_exec_props = config['configurations']['hive-exec-log4j']['content']
+else:
+  log4j_exec_props = None
+
+daemon_name = status_params.daemon_name
+process_name = status_params.process_name
+hive_env_sh_template = config['configurations']['hive-env']['content']
+
+hive_hdfs_user_dir = format("/user/{hive_user}")
+hive_hdfs_user_mode = 0755
+hive_apps_whs_dir = config['configurations']['hive-site']["hive.metastore.warehouse.dir"]
+whs_dir_protocol = urlparse(hive_apps_whs_dir).scheme
+hive_exec_scratchdir = config['configurations']['hive-site']["hive.exec.scratchdir"]
+#for create_hdfs_directory
+hdfs_user_keytab = config['configurations']['hadoop-env']['hdfs_user_keytab']
+hdfs_principal_name = default('/configurations/hadoop-env/hdfs_principal_name', 'missing_principal').replace("_HOST", hostname)
+
+# Tez-related properties
+tez_user = config['configurations']['tez-env']['tez_user']
+
+# Tez jars
+tez_local_api_jars = '/usr/lib/tez/tez*.jar'
+tez_local_lib_jars = '/usr/lib/tez/lib/*.jar'
+
+# Tez libraries
+tez_lib_uris = default("/configurations/tez-site/tez.lib.uris", None)
+
+if OSCheck.is_ubuntu_family():
+  mysql_configname = '/etc/mysql/my.cnf'
+else:
+  mysql_configname = '/etc/my.cnf'
+
+mysql_user = 'mysql'
+
+# Hive security
+hive_authorization_enabled = config['configurations']['hive-site']['hive.security.authorization.enabled']
+
+mysql_jdbc_driver_jar = "/usr/share/java/mysql-connector-java.jar"
+
+hive_site_config = dict(config['configurations']['hive-site'])
+
+########################################################
+############# AMS related params #####################
+########################################################
+ams_collector_hosts = default("/clusterHostInfo/metrics_collector_hosts", [])
+has_metric_collector = not len(ams_collector_hosts) == 0
+if has_metric_collector:
+  if 'cluster-env' in config['configurations'] and \
+      'metrics_collector_vip_host' in config['configurations']['cluster-env']:
+    metric_collector_host = config['configurations']['cluster-env']['metrics_collector_vip_host']
+  else:
+    metric_collector_host = ams_collector_hosts[0]
+  if 'cluster-env' in config['configurations'] and \
+      'metrics_collector_vip_port' in config['configurations']['cluster-env']:
+    metric_collector_port = config['configurations']['cluster-env']['metrics_collector_vip_port']
+  else:
+    metric_collector_web_address = default("/configurations/ams-site/timeline.metrics.service.webapp.address", "localhost:6188")
+    if metric_collector_web_address.find(':') != -1:
+      metric_collector_port = metric_collector_web_address.split(':')[1]
+    else:
+      metric_collector_port = '6188'
+  if default("/configurations/ams-site/timeline.metrics.service.http.policy", "HTTP_ONLY") == "HTTPS_ONLY":
+    metric_collector_protocol = 'https'
+  else:
+    metric_collector_protocol = 'http'
+  metric_truststore_path= default("/configurations/ams-ssl-client/ssl.client.truststore.location", "")
+  metric_truststore_type= default("/configurations/ams-ssl-client/ssl.client.truststore.type", "")
+  metric_truststore_password= default("/configurations/ams-ssl-client/ssl.client.truststore.password", "")
+
+metrics_report_interval = default("/configurations/ams-site/timeline.metrics.sink.report.interval", 60)
+metrics_collection_period = default("/configurations/ams-site/timeline.metrics.sink.collection.period", 10)
+
+########################################################
+############# Atlas related params #####################
+########################################################
+#region Atlas Hooks
+hive_atlas_application_properties = default('/configurations/hive-atlas-application.properties', {})
+
+if has_atlas_in_cluster():
+  atlas_hook_filename = default('/configurations/atlas-env/metadata_conf_file', 'atlas-application.properties')
+#endregion
+
+########################################################
+########### WebHCat related params #####################
+########################################################
+
+webhcat_env_sh_template = config['configurations']['webhcat-env']['content']
+templeton_log_dir = config['configurations']['hive-env']['hcat_log_dir']
+templeton_pid_dir = status_params.hcat_pid_dir
+
+webhcat_pid_file = status_params.webhcat_pid_file
+
+templeton_jar = config['configurations']['webhcat-site']['templeton.jar']
+
+
+webhcat_server_host = config['clusterHostInfo']['webhcat_server_host']
+
+hcat_hdfs_user_dir = format("/user/{hcat_user}")
+hcat_hdfs_user_mode = 0755
+webhcat_hdfs_user_dir = format("/user/{webhcat_user}")
+webhcat_hdfs_user_mode = 0755
+#for create_hdfs_directory
+security_param = "true" if security_enabled else "false"
+
+
+
+hdfs_site = config['configurations']['hdfs-site']
+default_fs = config['configurations']['core-site']['fs.defaultFS']
+
+dfs_type = default("/commandParams/dfs_type", "")
+
+import functools
+#create partial functions with common arguments for every HdfsResource call
+#to create hdfs directory we need to call params.HdfsResource in code
+HdfsResource = functools.partial(
+ HdfsResource,
+  user = hdfs_user,
+  hdfs_resource_ignore_file = "/var/lib/ambari-agent/data/.hdfs_resource_ignore",
+  security_enabled = security_enabled,
+  keytab = hdfs_user_keytab,
+  kinit_path_local = kinit_path_local,
+  hadoop_bin_dir = hadoop_bin_dir,
+  hadoop_conf_dir = hadoop_conf_dir,
+  principal_name = hdfs_principal_name,
+  hdfs_site = hdfs_site,
+  default_fs = default_fs,
+  immutable_paths = get_not_managed_resources(),
+  dfs_type = dfs_type
+ )
+
+# Hive Interactive related
+hive_interactive_hosts = default('/clusterHostInfo/hive_server_interactive_hosts', [])
+has_hive_interactive = len(hive_interactive_hosts) > 0
+if has_hive_interactive:
+  llap_daemon_log4j = config['configurations']['llap-daemon-log4j']['content']
+  llap_cli_log4j2 = config['configurations']['llap-cli-log4j2']['content']
+  hive_log4j2 = config['configurations']['hive-log4j2']['content']
+  hive_exec_log4j2 = config['configurations']['hive-exec-log4j2']['content']
+  beeline_log4j2 = config['configurations']['beeline-log4j2']['content']
+
+  hive_server_interactive_conf_dir = status_params.hive_server_interactive_conf_dir
+  execute_path_hive_interactive = os.path.join(os.environ['PATH'], hive_interactive_bin, hadoop_bin_dir)
+  start_hiveserver2_interactive_script = 'startHiveserver2Interactive.sh.j2'
+  start_hiveserver2_interactive_path = format("{tmp_dir}/start_hiveserver2_interactive_script")
+  hive_interactive_env_sh_template = config['configurations']['hive-interactive-env']['content']
+  hive_interactive_enabled = default('/configurations/hive-interactive-env/enable_hive_interactive', False)
+  llap_app_java_opts = default('/configurations/hive-interactive-env/llap_java_opts', '-XX:+AlwaysPreTouch {% if java_version > 7 %}-XX:+UseG1GC -XX:TLABSize=8m -XX:+ResizeTLAB -XX:+UseNUMA -XX:+AggressiveOpts -XX:MetaspaceSize=1024m -XX:InitiatingHeapOccupancyPercent=80 -XX:MaxGCPauseMillis=200{% else %}-XX:+PrintGCDetails -verbose:gc -XX:+PrintGCTimeStamps -XX:+UseNUMA -XX:+UseParallelGC{% endif %}')
+
+  # Service check related
+  if hive_transport_mode.lower() == "http":
+    hive_server_interactive_port = config['configurations']['hive-interactive-site']['hive.server2.thrift.http.port']
+  else:
+    hive_server_interactive_port = default('/configurations/hive-interactive-site/hive.server2.thrift.port',"10500")
+  # Tez for Hive interactive related
+  tez_interactive_config_dir = "/etc/tez_hive2/conf"
+  tez_interactive_user = config['configurations']['tez-env']['tez_user']
+  num_retries_for_checking_llap_status = default('/configurations/hive-interactive-env/num_retries_for_checking_llap_status', 10)
+  # Used in LLAP slider package creation
+  num_llap_nodes = config['configurations']['hive-interactive-env']['num_llap_nodes']
+  llap_daemon_container_size = config['configurations']['hive-interactive-site']['hive.llap.daemon.yarn.container.mb']
+  llap_log_level = config['configurations']['hive-interactive-env']['llap_log_level']
+  hive_llap_io_mem_size = config['configurations']['hive-interactive-site']['hive.llap.io.memory.size']
+  llap_heap_size = config['configurations']['hive-interactive-env']['llap_heap_size']
+  llap_app_name = config['configurations']['hive-interactive-env']['llap_app_name']
+  hive_llap_principal = None
+  if security_enabled:
+    hive_llap_keytab_file = config['configurations']['hive-interactive-site']['hive.llap.zk.sm.keytab.file']
+    hive_llap_principal = (config['configurations']['hive-interactive-site']['hive.llap.zk.sm.principal']).replace('_HOST',hostname.lower())
+  pass
+
+# ranger host
+ranger_admin_hosts = default("/clusterHostInfo/ranger_admin_hosts", [])
+has_ranger_admin = not len(ranger_admin_hosts) == 0
+xml_configurations_supported = config['configurations']['ranger-env']['xml_configurations_supported']
+
+#ranger hive properties
+policymgr_mgr_url = config['configurations']['admin-properties']['policymgr_external_url']
+if 'admin-properties' in config['configurations'] and 'policymgr_external_url' in config['configurations']['admin-properties'] and policymgr_mgr_url.endswith('/'):
+  policymgr_mgr_url = policymgr_mgr_url.rstrip('/')
+xa_audit_db_name = default('/configurations/admin-properties/audit_db_name', 'ranger_audits')
+xa_audit_db_user = default('/configurations/admin-properties/audit_db_user', 'rangerlogger')
+xa_db_host = config['configurations']['admin-properties']['db_host']
+repo_name = str(config['clusterName']) + '_hive'
+
+jdbc_driver_class_name = config['configurations']['ranger-hive-plugin-properties']['jdbc.driverClassName']
+common_name_for_certificate = config['configurations']['ranger-hive-plugin-properties']['common.name.for.certificate']
+
+repo_config_username = config['configurations']['ranger-hive-plugin-properties']['REPOSITORY_CONFIG_USERNAME']
+
+ranger_env = config['configurations']['ranger-env']
+ranger_plugin_properties = config['configurations']['ranger-hive-plugin-properties']
+policy_user = config['configurations']['ranger-hive-plugin-properties']['policy_user']
+
+if security_enabled:
+  hive_principal = hive_server_principal.replace('_HOST',hostname.lower())
+  hive_keytab = config['configurations']['hive-site']['hive.server2.authentication.kerberos.keytab']
+
+#For curl command in ranger plugin to get db connector
+if has_ranger_admin:
+  enable_ranger_hive = (config['configurations']['hive-env']['hive_security_authorization'].lower() == 'ranger')
+  repo_config_password = unicode(config['configurations']['ranger-hive-plugin-properties']['REPOSITORY_CONFIG_PASSWORD'])
+  xa_audit_db_flavor = (config['configurations']['admin-properties']['DB_FLAVOR']).lower()
+  ranger_previous_jdbc_jar_name = None
+
+  if stack_supports_ranger_audit_db:
+    if xa_audit_db_flavor and xa_audit_db_flavor == 'mysql':
+      ranger_jdbc_jar_name = default("/hostLevelParams/custom_mysql_jdbc_name", None)
+      ranger_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_mysql_jdbc_name", None)
+      audit_jdbc_url = format('jdbc:mysql://{xa_db_host}/{xa_audit_db_name}')
+      jdbc_driver = "com.mysql.jdbc.Driver"
+    elif xa_audit_db_flavor and xa_audit_db_flavor == 'oracle':
+      ranger_jdbc_jar_name = default("/hostLevelParams/custom_oracle_jdbc_name", None)
+      ranger_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_oracle_jdbc_name", None)
+      colon_count = xa_db_host.count(':')
+      if colon_count == 2 or colon_count == 0:
+        audit_jdbc_url = format('jdbc:oracle:thin:@{xa_db_host}')
+      else:
+        audit_jdbc_url = format('jdbc:oracle:thin:@//{xa_db_host}')
+      jdbc_driver = "oracle.jdbc.OracleDriver"
+    elif xa_audit_db_flavor and xa_audit_db_flavor == 'postgres':
+      ranger_jdbc_jar_name = default("/hostLevelParams/custom_postgres_jdbc_name", None)
+      ranger_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_postgres_jdbc_name", None)
+      audit_jdbc_url = format('jdbc:postgresql://{xa_db_host}/{xa_audit_db_name}')
+      jdbc_driver = "org.postgresql.Driver"
+    elif xa_audit_db_flavor and xa_audit_db_flavor == 'mssql':
+      ranger_jdbc_jar_name = default("/hostLevelParams/custom_mssql_jdbc_name", None)
+      ranger_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_mssql_jdbc_name", None)
+      audit_jdbc_url = format('jdbc:sqlserver://{xa_db_host};databaseName={xa_audit_db_name}')
+      jdbc_driver = "com.microsoft.sqlserver.jdbc.SQLServerDriver"
+    elif xa_audit_db_flavor and xa_audit_db_flavor == 'sqla':
+      ranger_jdbc_jar_name = default("/hostLevelParams/custom_sqlanywhere_jdbc_name", None)
+      ranger_previous_jdbc_jar_name = default("/hostLevelParams/previous_custom_sqlanywhere_jdbc_name", None)
+      audit_jdbc_url = format('jdbc:sqlanywhere:database={xa_audit_db_name};host={xa_db_host}')
+      jdbc_driver = "sap.jdbc4.sqlanywhere.IDriver"
+
+  ranger_downloaded_custom_connector = format("{tmp_dir}/{ranger_jdbc_jar_name}") if stack_supports_ranger_audit_db else None
+  ranger_driver_curl_source = format("{jdk_location}/{ranger_jdbc_jar_name}") if stack_supports_ranger_audit_db else None
+  ranger_driver_curl_target = format("{hive_lib}/{ranger_jdbc_jar_name}") if stack_supports_ranger_audit_db else None
+  ranger_previous_jdbc_jar = format("{hive_lib}/{ranger_previous_jdbc_jar_name}") if stack_supports_ranger_audit_db else None
+  sql_connector_jar = ''
+
+  hive_ranger_plugin_config = {
+    'username': repo_config_username,
+    'password': repo_config_password,
+    'jdbc.driverClassName': jdbc_driver_class_name,
+    'jdbc.url': format("{hive_url}/default;principal={hive_principal}") if security_enabled else hive_url,
+    'commonNameForCertificate': common_name_for_certificate
+  }
+
+  hive_ranger_plugin_repo = {
+    'isActive': 'true',
+    'config': json.dumps(hive_ranger_plugin_config),
+    'description': 'hive repo',
+    'name': repo_name,
+    'repositoryType': 'hive',
+    'assetType': '3'
+  }
+
+  if stack_supports_ranger_kerberos and security_enabled:
+    hive_ranger_plugin_config['policy.download.auth.users'] = hive_user
+    hive_ranger_plugin_config['tag.download.auth.users'] = hive_user
+    hive_ranger_plugin_config['policy.grantrevoke.auth.users'] = hive_user
+
+  if stack_supports_ranger_kerberos:
+    hive_ranger_plugin_config['ambari.service.check.user'] = policy_user
+
+    hive_ranger_plugin_repo = {
+      'isEnabled': 'true',
+      'configs': hive_ranger_plugin_config,
+      'description': 'hive repo',
+      'name': repo_name,
+      'type': 'hive'
+    }
+
+
+  xa_audit_db_is_enabled = False
+  xa_audit_db_password = ''
+  if not is_empty(config['configurations']['admin-properties']['audit_db_password']) and stack_supports_ranger_audit_db:
+    xa_audit_db_password = unicode(config['configurations']['admin-properties']['audit_db_password'])
+  ranger_audit_solr_urls = config['configurations']['ranger-admin-site']['ranger.audit.solr.urls']
+  if xml_configurations_supported and stack_supports_ranger_audit_db:
+    xa_audit_db_is_enabled = config['configurations']['ranger-hive-audit']['xasecure.audit.destination.db']
+  xa_audit_hdfs_is_enabled = config['configurations']['ranger-hive-audit']['xasecure.audit.destination.hdfs'] if xml_configurations_supported else None
+  ssl_keystore_password = unicode(config['configurations']['ranger-hive-policymgr-ssl']['xasecure.policymgr.clientssl.keystore.password']) if xml_configurations_supported else None
+  ssl_truststore_password = unicode(config['configurations']['ranger-hive-policymgr-ssl']['xasecure.policymgr.clientssl.truststore.password']) if xml_configurations_supported else None
+  credential_file = format('/etc/ranger/{repo_name}/cred.jceks') if xml_configurations_supported else None
+
+  #For SQLA explicitly disable audit to DB for Ranger
+  if xa_audit_db_flavor == 'sqla':
+    xa_audit_db_is_enabled = False
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/ba8d7f50/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_windows.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_windows.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_windows.py
new file mode 100755
index 0000000..880fdb5
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/params_windows.py
@@ -0,0 +1,74 @@
+#!/usr/bin/env python
+"""
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+"""
+
+from resource_management import *
+from status_params import *
+
+# server configurations
+config = Script.get_config()
+
+# This is expected to be of the form #.#.#.#
+stack_version_unformatted = config['hostLevelParams']['stack_version']
+stack_version_formatted = format_stack_version(stack_version_unformatted)
+
+stack_root = None
+hive_conf_dir = None
+hive_home = None
+hive_lib_dir = None
+hive_log_dir = None
+hive_opts = None
+hcat_home = None
+hcat_config_dir = None
+hive_bin = None
+
+try:
+  stack_root = os.path.abspath(os.path.join(os.environ["HADOOP_HOME"],".."))
+  hive_conf_dir = os.environ["HIVE_CONF_DIR"]
+  hive_home = os.environ["HIVE_HOME"]
+  hive_lib_dir = os.environ["HIVE_LIB_DIR"]
+  hive_log_dir = os.environ["HIVE_LOG_DIR"]
+  hive_opts = os.environ["HIVE_OPTS"]
+  hcat_home = os.environ["HCAT_HOME"]
+  hcat_config_dir = os.environ["WEBHCAT_CONF_DIR"]
+  hive_bin = os.path.join(hive_home, "bin")
+except:
+  pass
+
+hive_env_sh_template = config['configurations']['hive-env']['content']
+hive_warehouse_dir = config['configurations']['hive-site']['hive.metastore.warehouse.dir']
+hadoop_user = config["configurations"]["cluster-env"]["hadoop.user.name"]
+hive_user = hadoop_user
+hcat_user = hadoop_user
+
+hive_metastore_db_type = config['configurations']['hive-env']['hive_database_type']
+hive_metastore_user_name = config['configurations']['hive-site']['javax.jdo.option.ConnectionUserName']
+hive_metastore_user_passwd = config['configurations']['hive-site']['javax.jdo.option.ConnectionPassword']
+
+hive_execution_engine = config["configurations"]["hive-site"]["hive.execution.engine"]
+
+######## Metastore Schema
+init_metastore_schema = not config['configurations']['hive-site']['datanucleus.autoCreateSchema']
+
+service_map = {
+  "metastore" : hive_metastore_win_service_name,
+  "client" : hive_client_win_service_name,
+  "hiveserver2" : hive_server_win_service_name,
+  "templeton" : webhcat_server_win_service_name
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/ba8d7f50/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/service_check.py b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/service_check.py
new file mode 100755
index 0000000..1836d0f
--- /dev/null
+++ b/bigtop-packages/src/common/ambari/ODPi/1.0/services/HIVE/package/scripts/service_check.py
@@ -0,0 +1,190 @@
+#!/usr/bin/env python
+"""
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+"""
+
+from resource_management import *
+import socket
+import sys
+import time
+import subprocess
+
+from hcat_service_check import hcat_service_check
+from webhcat_service_check import webhcat_service_check
+from ambari_commons import OSConst
+from ambari_commons.os_family_impl import OsFamilyImpl
+from resource_management.core import shell
+from resource_management.core.logger import Logger
+from resource_management.libraries.functions import get_unique_id_and_date
+
+class HiveServiceCheck(Script):
+  pass
+
+
+@OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
+class HiveServiceCheckWindows(HiveServiceCheck):
+  def service_check(self, env):
+    import params
+    env.set_params(params)
+    smoke_cmd = os.path.join(params.stack_root,"Run-SmokeTests.cmd")
+    service = "HIVE"
+    Execute(format("cmd /C {smoke_cmd} {service}"), user=params.hive_user, logoutput=True)
+
+    hcat_service_check()
+    webhcat_service_check()
+
+
+@OsFamilyImpl(os_family=OsFamilyImpl.DEFAULT)
+class HiveServiceCheckDefault(HiveServiceCheck):
+
+  def __init__(self):
+    super(HiveServiceCheckDefault, self).__init__()
+    Logger.initialize_logger()
+
+  def service_check(self, env):
+    import params
+    env.set_params(params)
+
+    if params.security_enabled:
+      kinit_cmd = format(
+        "{kinit_path_local} -kt {smoke_user_keytab} {smokeuser_principal}; ")
+    else:
+      kinit_cmd = ""
+
+    # Check HiveServer
+    Logger.info("Running Hive Server checks")
+    Logger.info("--------------------------\n")
+    self.check_hive_server(env, 'Hive Server', kinit_cmd, params.hive_server_hosts,
+                           int(format("{hive_server_port}")))
+
+
+    if params.has_hive_interactive  and params.hive_interactive_enabled:
+      Logger.info("Running Hive Server2 checks")
+      Logger.info("--------------------------\n")
+
+      self.check_hive_server(env, 'Hive Server2', kinit_cmd, params.hive_interactive_hosts,
+                             int(format("{hive_server_interactive_port}")))
+
+      Logger.info("Running LLAP checks")
+      Logger.info("-------------------\n")
+      self.check_llap(env, kinit_cmd, params.hive_interactive_hosts, int(format("{hive_server_interactive_port}")),
+                      params.hive_llap_principal, params.hive_server2_authentication, params.hive_transport_mode,
+                      params.hive_http_endpoint)
+
+
+    Logger.info("Running HCAT checks")
+    Logger.info("-------------------\n")
+    hcat_service_check()
+
+    Logger.info("Running WEBHCAT checks")
+    Logger.info("---------------------\n")
+    webhcat_service_check()
+
+  def check_hive_server(self, env, server_component_name, kinit_cmd, address_list, server_port):
+    import params
+    env.set_params(params)
+    Logger.info("Server Address List : {0}, Port : {1}".format(address_list, server_port))
+
+    if not address_list:
+      raise Fail("Can not find any "+server_component_name+" ,host. Please check configuration.")
+
+    SOCKET_WAIT_SECONDS = 290
+
+    start_time = time.time()
+    end_time = start_time + SOCKET_WAIT_SECONDS
+
+    Logger.info("Waiting for the {0} to start...".format(server_component_name))
+
+    workable_server_available = False
+    i = 0
+    while time.time() < end_time and not workable_server_available:
+      address = address_list[i]
+      try:
+        check_thrift_port_sasl(address, server_port, params.hive_server2_authentication,
+                               params.hive_server_principal, kinit_cmd, params.smokeuser,
+                               transport_mode=params.hive_transport_mode, http_endpoint=params.hive_http_endpoint,
+                               ssl=params.hive_ssl, ssl_keystore=params.hive_ssl_keystore_path,
+                               ssl_password=params.hive_ssl_keystore_password)
+        Logger.info("Successfully connected to {0} on port {1}".format(address, server_port))
+        workable_server_available = True
+      except:
+        Logger.info("Connection to {0} on port {1} failed".format(address, server_port))
+        time.sleep(5)
+
+      i += 1
+      if i == len(address_list):
+        i = 0
+
+    elapsed_time = time.time() - start_time
+
+    if not workable_server_available:
+      raise Fail("Connection to '{0}' on host: {1} and port {2} failed after {3} seconds"
+                 .format(server_component_name, params.hostname, server_port, elapsed_time))
+
+    Logger.info("Successfully stayed connected to '{0}' on host: {1} and port {2} after {3} seconds"
+                .format(server_component_name, params.hostname, server_port, elapsed_time))
+
+  """
+  Performs Service check for LLAP app
+  """
+  def check_llap(self, env, kinit_cmd, address, port, key, hive_auth="NOSASL", transport_mode="binary", http_endpoint="cliservice"):
+    import params
+    env.set_params(params)
+
+    unique_id = get_unique_id_and_date()
+
+    beeline_url = ['jdbc:hive2://{address}:{port}/', "transportMode={transport_mode}"]
+
+    # Currently, HSI is supported on a single node only. The address list should be of size 1,
+    # thus picking the 1st node value.
+    address = address[0]
+
+    # append url according to used transport
+    if transport_mode == "http":
+      beeline_url.append('httpPath={http_endpoint}')
+
+    # append url according to used auth
+    if hive_auth == "NOSASL":
+      beeline_url.append('auth=noSasl')
+
+    # append url according to principal
+    if kinit_cmd:
+      beeline_url.append('principal={key}')
+
+    exec_path = params.execute_path
+    if params.version and params.stack_root:
+      upgrade_hive_bin = format("{stack_root}/{version}/hive2/bin")
+      exec_path =  os.environ['PATH'] + os.pathsep + params.hadoop_bin_dir + os.pathsep + upgrade_hive_bin
+
+    # beeline path
+    llap_cmd = "! beeline -u '%s'" % format(";".join(beeline_url))
+    # Append LLAP SQL script path
+    llap_cmd += format(" --hiveconf \"hiveLlapServiceCheck={unique_id}\" -f {stack_root}/current/hive-server2-hive2/scripts/llap/sql/serviceCheckScript.sql")
+    # Append grep patterns for detecting failure
+    llap_cmd += " -e '' 2>&1| awk '{print}'|grep -i -e 'Invalid status\|Invalid URL\|command not found\|Connection refused'"
+
+    Execute(llap_cmd,
+            user=params.hive_user,
+            path=['/usr/sbin', '/usr/local/bin', '/bin', '/usr/bin', exec_path],
+            tries=1,
+            wait_for_finish=True,
+            stderr=subprocess.PIPE,
+            logoutput=True)
+
+if __name__ == "__main__":
+  HiveServiceCheck().execute()
\ No newline at end of file