You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by sm...@apache.org on 2016/03/28 21:55:53 UTC

[2/2] ambari git commit: Ambari-15573. Add the Config, Start and Stop logic for Hive Server Interactive, and Associate the LLAP lifecycle to it (Swapan Sridhar via smohanty)

Ambari-15573. Add the Config, Start and Stop logic for Hive Server Interactive, and  Associate the LLAP lifecycle to it (Swapan Sridhar via smohanty)


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

Branch: refs/heads/trunk
Commit: d4ab6fdf8c38ec2113f8ccef079dc4e4627ad9a7
Parents: 4664676
Author: Sumit Mohanty <sm...@hortonworks.com>
Authored: Mon Mar 28 12:55:36 2016 -0700
Committer: Sumit Mohanty <sm...@hortonworks.com>
Committed: Mon Mar 28 12:55:36 2016 -0700

----------------------------------------------------------------------
 .../libraries/functions/copy_tarball.py         |    8 +-
 .../HIVE/0.12.0.2.0/package/scripts/hive.py     |   11 +-
 .../package/scripts/hive_interactive.py         |  124 ++
 .../package/scripts/hive_server_interactive.py  |   46 +-
 .../0.12.0.2.0/package/scripts/hive_service.py  |   10 +-
 .../package/scripts/hive_service_interactive.py |  175 ++
 .../0.12.0.2.0/package/scripts/params_linux.py  |   23 +-
 .../0.12.0.2.0/package/scripts/status_params.py |    4 +-
 .../templates/startHiveserver2Interactive.sh.j2 |   24 +
 .../HDP/2.0.6/properties/tarball_map.json       |    8 +
 .../configuration/hive-interactive-site.xml     | 2009 ++----------------
 11 files changed, 533 insertions(+), 1909 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/d4ab6fdf/ambari-common/src/main/python/resource_management/libraries/functions/copy_tarball.py
----------------------------------------------------------------------
diff --git a/ambari-common/src/main/python/resource_management/libraries/functions/copy_tarball.py b/ambari-common/src/main/python/resource_management/libraries/functions/copy_tarball.py
index f07b76f..2ba12b6 100644
--- a/ambari-common/src/main/python/resource_management/libraries/functions/copy_tarball.py
+++ b/ambari-common/src/main/python/resource_management/libraries/functions/copy_tarball.py
@@ -42,8 +42,12 @@ _DEFAULT_TARBALL_MAP = {
              "/{0}/apps/{1}/slider/slider.tar.gz".format(STACK_NAME_PATTERN, STACK_VERSION_PATTERN)),
   "tez": ("{0}/{1}/tez/lib/tez.tar.gz".format(STACK_ROOT_PATTERN, STACK_VERSION_PATTERN),
           "/{0}/apps/{1}/tez/tez.tar.gz".format(STACK_NAME_PATTERN, STACK_VERSION_PATTERN)),
+  "tez_hive2": ("{0}/{1}/tez_hive2/lib/tez.tar.gz".format(STACK_ROOT_PATTERN, STACK_VERSION_PATTERN),
+          "/{0}/apps/{1}/tez_hive2/tez.tar.gz".format(STACK_NAME_PATTERN, STACK_VERSION_PATTERN)),
   "hive": ("{0}/{1}/hive/hive.tar.gz".format(STACK_ROOT_PATTERN, STACK_VERSION_PATTERN),
            "/{0}/apps/{1}/hive/hive.tar.gz".format(STACK_NAME_PATTERN, STACK_VERSION_PATTERN)),
+  "hive2": ("{0}/{1}/hive2/hive.tar.gz".format(STACK_ROOT_PATTERN, STACK_VERSION_PATTERN),
+           "/{0}/apps/{1}/hive2/hive.tar.gz".format(STACK_NAME_PATTERN, STACK_VERSION_PATTERN)),
   "pig": ("{0}/{1}/pig/pig.tar.gz".format(STACK_ROOT_PATTERN, STACK_VERSION_PATTERN),
           "/{0}/apps/{1}/pig/pig.tar.gz".format(STACK_NAME_PATTERN, STACK_VERSION_PATTERN)),
   "hadoop_streaming": ("{0}/{1}/hadoop-mapreduce/hadoop-streaming.jar".format(STACK_ROOT_PATTERN, STACK_VERSION_PATTERN),
@@ -76,7 +80,6 @@ def _get_tarball_paths(name, use_upgrading_version_during_uprade=True, custom_so
     Logger.error("Cannot copy {0} tarball to HDFS because stack name could be be determined.".format(
             str(name)))
     return (False, None, None)
-
   stack_version = _get_current_version(use_upgrading_version_during_uprade)
   if not stack_version:
     Logger.error("Cannot copy {0} tarball to HDFS because stack version could be be determined.".format(
@@ -98,7 +101,6 @@ def _get_tarball_paths(name, use_upgrading_version_during_uprade=True, custom_so
     Logger.error("Cannot copy tarball to HDFS because {0} is not supported in stack {1} for this operation.".format(
             str(name), str(stack_name)))
     return (False, None, None)
-
   (source_file, dest_file) = tarball_map[name.lower()]
 
   if custom_source_file is not None:
@@ -106,7 +108,6 @@ def _get_tarball_paths(name, use_upgrading_version_during_uprade=True, custom_so
 
   if custom_dest_file is not None:
     dest_file = custom_dest_file
-
   source_file = source_file.replace(STACK_NAME_PATTERN, stack_name.lower())
   dest_file = dest_file.replace(STACK_NAME_PATTERN, stack_name.lower())
 
@@ -115,6 +116,7 @@ def _get_tarball_paths(name, use_upgrading_version_during_uprade=True, custom_so
 
   source_file = source_file.replace(STACK_VERSION_PATTERN, stack_version)
   dest_file = dest_file.replace(STACK_VERSION_PATTERN, stack_version)
+
   return (True, source_file, dest_file)
 
 def _get_current_version(use_upgrading_version_during_uprade=True):

http://git-wip-us.apache.org/repos/asf/ambari/blob/d4ab6fdf/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive.py
index 3532efc..3c51d39 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive.py
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive.py
@@ -302,10 +302,13 @@ def hive(name=None):
     )
 
   if name != "client":
-    crt_directory(params.hive_pid_dir)
-    crt_directory(params.hive_log_dir)
-    crt_directory(params.hive_var_lib)
+    create_directory(params.hive_pid_dir)
+    create_directory(params.hive_log_dir)
+    create_directory(params.hive_var_lib)
 
+"""
+Writes configuration files required by Hive.
+"""
 def fill_conf_dir(component_conf_dir):
   import params
   
@@ -359,7 +362,7 @@ def fill_conf_dir(component_conf_dir):
          content=StaticFile(format("{component_conf_dir}/{log4j_filename}.template"))
     )
 
-def crt_directory(name):
+def create_directory(name):
   import params
 
   Directory(name,

http://git-wip-us.apache.org/repos/asf/ambari/blob/d4ab6fdf/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_interactive.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_interactive.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_interactive.py
new file mode 100644
index 0000000..188f199
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_interactive.py
@@ -0,0 +1,124 @@
+#!/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 glob
+from urlparse import urlparse
+
+from resource_management.libraries.script.script import Script
+from resource_management.libraries.resources.hdfs_resource import HdfsResource
+from resource_management.libraries.functions.copy_tarball import copy_to_hdfs
+from resource_management.libraries.functions.version import compare_versions
+from resource_management.core.resources.service import ServiceConfig
+from resource_management.core.resources.system import File, Execute, Directory
+from resource_management.core.source import StaticFile, Template, DownloadSource, InlineTemplate
+from resource_management.core.shell import as_user
+from resource_management.libraries.functions.is_empty import is_empty
+from resource_management.libraries.resources.xml_config import XmlConfig
+from resource_management.libraries.functions.format import format
+from resource_management.core.exceptions import Fail
+from resource_management.core.shell import as_sudo
+from resource_management.core.shell import quote_bash_args
+from resource_management.core.logger import Logger
+from resource_management.core import utils
+
+from ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from ambari_commons import OSConst
+from hive import fill_conf_dir, create_directory, jdbc_connector
+
+
+@OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
+def hive_interactive(name=None):
+  pass
+
+"""
+Sets up the configs, jdbc connection and tarball copy to HDFS for Hive Server Interactive.
+"""
+@OsFamilyFuncImpl(os_family=OsFamilyImpl.DEFAULT)
+def hive_interactive(name=None):
+  import params
+
+  # Copy Tarballs in HDFS.
+  copy_to_hdfs("tez_hive2",
+               params.user_group,
+               params.hdfs_user,
+               file_mode=params.tarballs_mode,
+               host_sys_prepped=params.host_sys_prepped)
+
+  copy_to_hdfs("hive2",
+               params.user_group,
+               params.hdfs_user,
+               file_mode=params.tarballs_mode,
+               host_sys_prepped=params.host_sys_prepped)
+
+  params.HdfsResource(None, action="execute")
+
+  Directory(params.hive_interactive_etc_dir_prefix,
+            mode=0755
+            )
+
+  for conf_dir in params.hive_interactive_conf_dir:
+    fill_conf_dir(conf_dir)
+
+  '''
+  As hive2/hive-site.xml only contains the new + the changed props compared to hive/hive-site.xml,
+  we need to merge hive/hive-site.xml and hive2/hive-site.xml and store it in hive2/hive-site.xml.
+  '''
+  merged_hive_interactive_site = {}
+  merged_hive_interactive_site.update(params.config['configurations']['hive-site'])
+  merged_hive_interactive_site.update(params.config['configurations']['hive-interactive-site'])
+
+  XmlConfig("hive-site.xml",
+            conf_dir=params.hive_server_interactive_conf_dir,
+            configurations=merged_hive_interactive_site,
+            configuration_attributes=params.config['configuration_attributes']['hive-interactive-site'],
+            owner=params.hive_user,
+            group=params.user_group,
+            mode=0644)
+
+  XmlConfig("tez-site.xml",
+             conf_dir = params.tez_interactive_config_dir,
+             configurations = params.config['configurations']['tez-interactive-site'],
+             configuration_attributes=params.config['configuration_attributes']['tez-interactive-site'],
+             owner = params.tez_interactive_user,
+             group = params.user_group,
+             mode = 0664)
+
+  File(format("{hive_server_interactive_conf_dir}/hive-env.sh"),
+       owner=params.hive_user,
+       group=params.user_group,
+       content=InlineTemplate(params.hive_interactive_env_sh_template)
+       )
+
+  if not os.path.exists(params.target_hive_interactive):
+    jdbc_connector()
+
+  File(format("/usr/lib/ambari-agent/{check_db_connection_jar_name}"),
+       content = DownloadSource(format("{jdk_location}{check_db_connection_jar_name}")),
+       mode = 0644,
+       )
+  File(params.start_hiveserver2_interactive_path,
+       mode=0755,
+       content=Template(format('{start_hiveserver2_interactive_script}'))
+       )
+
+  create_directory(params.hive_pid_dir)
+  create_directory(params.hive_log_dir)
+  create_directory(params.hive_interactive_var_lib)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/d4ab6fdf/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_server_interactive.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_server_interactive.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_server_interactive.py
index 8a4e7e6..e1c0c42 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_server_interactive.py
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_server_interactive.py
@@ -31,13 +31,16 @@ from resource_management.libraries.functions.version import compare_versions, fo
 from resource_management.libraries.functions.security_commons import build_expectations, \
     cached_kinit_executor, get_params_from_filesystem, validate_security_config_properties, \
     FILE_TYPE_XML
+from resource_management.core.resources.system import File, Execute
+from resource_management.core import shell
+from subprocess import call
 from ambari_commons import OSCheck, OSConst
 from setup_ranger_hive import setup_ranger_hive
 from ambari_commons.os_family_impl import OsFamilyImpl
 from ambari_commons.constants import UPGRADE_TYPE_ROLLING
 from resource_management.core.logger import Logger
-
-import hive_server_upgrade
+from hive_service_interactive import hive_service_interactive, stop_llap, start_llap
+from hive_interactive import hive_interactive
 
 class HiveServerInteractive(Script):
     def install(self, env):
@@ -45,7 +48,9 @@ class HiveServerInteractive(Script):
       self.install_packages(env)
 
     def configure(self, env):
-      pass
+      import params
+      env.set_params(params)
+      hive_interactive(name='hiveserver2')
 
 @OsFamilyImpl(os_family=OSConst.WINSRV_FAMILY)
 class HiveServerWindows(HiveServerInteractive):
@@ -65,15 +70,44 @@ class HiveServerDefault(HiveServerInteractive):
       return {"HDP": "hive-server2-hive2"}
 
     def start(self, env, upgrade_type=None):
-      pass
+      import params
+      env.set_params(params)
+      self.configure(env)
+
+      # TODO : We need have conditional [re]start of LLAP once "status check command" for LLAP is ready.
+      # Check status and based on that decide on [re]starting.
+
+      # Start LLAP before Hive Server Interactive start
+      status = start_llap(self)
+      if status:
+        # TODO : test the workability of Ranger and Hive2 during upgrade
+        # setup_ranger_hive(upgrade_type=upgrade_type)
+
+        hive_service_interactive('hiveserver2', action='start', upgrade_type=upgrade_type)
+      else:
+        Logger.info("Skipping start of Hive Server Interactive due to LLAP start issue.")
 
     def stop(self, env, upgrade_type=None):
-      pass
+      import params
+      env.set_params(params)
+
+      # Stop Hive Interactive Server first
+      # TODO : Upgrade check comes here.
+      hive_service_interactive('hiveserver2', action = 'stop')
+
+      stop_llap(self)
 
     def status(self, env):
-      pass
+      import status_params
+      env.set_params(status_params)
+      pid_file = format("{hive_pid_dir}/{hive_interactive_pid}")
+
+      # Recursively check all existing gmetad pid files
+      check_process_status(pid_file)
+      # TODO : Check the LLAP app status as well.
 
     def pre_upgrade_restart(self, env, upgrade_type=None):
+      # TODO: Make sure, the tez_hive2 is upgraded, while writing the upgrade code.
       pass
 
     def security_status(self, env):

http://git-wip-us.apache.org/repos/asf/ambari/blob/d4ab6fdf/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_service.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_service.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_service.py
index 12cf336..2f94dfb 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_service.py
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_service.py
@@ -70,7 +70,7 @@ def hive_service(name, action='start', upgrade_type=None):
 
   if action == 'start':
     if name == 'hiveserver2':
-      check_fs_root()
+      check_fs_root(params.hive_server_conf_dir, params.execute_path)
 
     daemon_cmd = cmd
     hadoop_home = params.hadoop_home
@@ -126,15 +126,15 @@ def hive_service(name, action='start', upgrade_type=None):
          action = "delete"
     )
 
-def check_fs_root():
+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 {hive_server_conf_dir} --service metatool")
-  cmd = as_user(format("{metatool_cmd} -listFSRoot", env={'PATH': params.execute_path}), params.hive_user) \
+  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)
 
@@ -143,6 +143,6 @@ def check_fs_root():
     cmd = format("{metatool_cmd} -updateLocation {fs_root} {out}")
     Execute(cmd,
             user=params.hive_user,
-            environment={'PATH': params.execute_path}
+            environment={'PATH': execution_path}
     )
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/d4ab6fdf/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_service_interactive.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_service_interactive.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_service_interactive.py
new file mode 100644
index 0000000..9f32d16
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_service_interactive.py
@@ -0,0 +1,175 @@
+#!/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
+import sys
+import shutil
+import subprocess
+
+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 ambari_commons.os_family_impl import OsFamilyFuncImpl, OsFamilyImpl
+from ambari_commons import OSConst
+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}")
+
+  # TODO : Kerberos work for Hive2
+
+  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"
+
+    # TODO : Upgrade checks required here.
+
+    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":
+      db_connection_check_command = format(
+        "{java64_home}/bin/java -cp {check_db_connection_jar}:{target} 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})")
+            )
+
+    wait_time = 5
+    Execute(daemon_hard_kill_cmd,
+            not_if = format("! ({process_id_exists_command}) || ( sleep {wait_time} && ! ({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"
+         )
+
+
+def start_llap(self):
+  import params
+  abs_dir_path = ''
+  try:
+    # TODO : Currently hardcoded the params. Need to read the suggested values from hive2/hive-site.xml.
+    cmd = ["/usr/hdp/current/hive-server2-hive2/bin/hive", "--service", "llap", "--instances", "1",
+           "-slider-am-container-mb", "341", "--loglevel", "INFO"]
+    code, output, error = shell.checked_call(cmd, user=params.hive_user, stderr=subprocess.PIPE)
+
+    # TODO : ideally we should check error status, but currently LLAP package creation throws 'ClassNotFoundException'
+    # for 'JsonSerDe', but still goes ahead and creates the package which works.
+    if output is not None:
+      # Expected 'output' string is of the form : "Prepared llap-slider-[DDMMYYYY]/run.sh for running LLAP on Slider"
+      Logger.info("LLAP package creation output : {0}".format(output))
+      splits = output.split()
+      if len(splits) > 2:
+        if "llap-slider-" in splits[1]:
+          llap_dir, llap_run_file = (splits[1]).split("/")
+          abs_dir_path = os.path.join(params.hive_user_home_dir, llap_dir)
+          run_file_abs_path = os.path.join(abs_dir_path, llap_run_file)
+          file_exists = os.path.isfile(run_file_abs_path)
+          if file_exists:
+            Execute(run_file_abs_path, user=params.hive_user)
+            # TODO : Sleep below is not a good idea. We need to check the status of LLAP app to figure out it got
+            # launched properly and is in running state. Then go ahead with Hive Interactive Server start.
+            time.sleep(30)
+            Logger.info("LLAP app deployed successfully.")
+            return True
+          else:
+            Logger.error("LLAP slider package : {0} , not present in path : {1}. Exiting ... ".format(llap_dir,
+                                                                                                      params.hive_user_home_dir))
+            return False
+      else:
+        # Looks like assumption of successful/correct output string being "Prepared llap-slider-[DDMMYYYY]/run.sh
+        # for running LLAP on Slider" has changed.
+        Logger.error("Couldn't parse the message {0} for LLAP slider package. Exiting ... ".format(output))
+        return False
+    else:
+      Logger.error(
+        "Error while creating the LLAP slider package. \n Error Code : {0} \n Output : {1}".format(error, output))
+      return False
+
+  except:
+    Logger.error("Error: {0}".format(sys.exc_info()))
+    return False
+  finally:
+    # Do the cleanup
+    dir_exists = os.path.isdir(abs_dir_path)
+    if dir_exists:
+      shutil.rmtree(abs_dir_path)
+
+
+def stop_llap(self):
+  import params
+  try:
+    stop_cmd = ("slider", "stop", "llap0")
+    print "STOP cmd  = ",stop_cmd
+    Execute(stop_cmd, user=params.hive_user, timeout=30, wait_for_finish=True)
+
+    # TODO : Check status of LLAP app as STOPPED/FINSIHED, before destroying.
+    destroy_cmd = ['slider', 'destroy', 'llap0']
+    code, output, error = shell.checked_call(destroy_cmd, user=params.hive_user, stderr=subprocess.PIPE)
+    if error is None or not error:
+      Logger.info("Removed slider app : llap0.")
+    else:
+      Logger.error("Problem removing slider app : llap0. Exiting ....")
+
+  except:
+    Logger.info("Error: {0}".format(sys.exc_info()))

http://git-wip-us.apache.org/repos/asf/ambari/blob/d4ab6fdf/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params_linux.py
index 6f05fbc..481b588 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params_linux.py
@@ -76,6 +76,7 @@ upgrade_direction = default("/commandParams/upgrade_direction", None)
 
 hadoop_bin_dir = "/usr/bin"
 hadoop_home = '/usr'
+hive_user_home_dir = "/home/hive"
 hive_bin = '/usr/lib/hive/bin'
 hive_lib = '/usr/lib/hive/lib/'
 hive_var_lib = '/var/lib/hive'
@@ -110,6 +111,7 @@ hive_client_conf_dir = status_params.hive_client_conf_dir
 hive_server_conf_dir = status_params.hive_server_conf_dir
 hive_interactive_conf_dir = status_params.hive_server_interactive_conf_dir
 
+
 hcat_conf_dir = '/etc/hive-hcatalog/conf'
 config_dir = '/etc/hive-webhcat/conf'
 hcat_lib = '/usr/lib/hive-hcatalog/share/hcatalog'
@@ -173,6 +175,7 @@ else:
   tarballs_mode = 0755
 
 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']
 
@@ -310,7 +313,7 @@ artifact_dir = format("{tmp_dir}/AMBARI-artifacts/")
 yarn_log_dir_prefix = config['configurations']['yarn-env']['yarn_log_dir_prefix']
 
 target = format("{hive_lib}/{jdbc_jar_name}")
-target_interactive = format("{hive_interactive_lib}/{jdbc_jar_name}")
+target_hive_interactive = format("{hive_interactive_lib}/{jdbc_jar_name}")
 jars_in_hive_lib = format("{hive_lib}/*.jar")
 
 start_hiveserver2_path = format("{tmp_dir}/start_hiveserver2_script")
@@ -407,11 +410,6 @@ mysql_jdbc_driver_jar = "/usr/share/java/mysql-connector-java.jar"
 
 hive_site_config = dict(config['configurations']['hive-site'])
 
-hive_interactive_hosts = default('/clusterHostInfo/hive-server2-hive2_hosts', [])
-has_hive_interactive = len(hive_interactive_hosts) > 0
-if has_hive_interactive:
-  hive_interactive_site_config = dict(config['configurations']['hive-interactive-site'])
-
 ########################################################
 ############# Atlas related params #####################
 ########################################################
@@ -481,6 +479,19 @@ HdfsResource = functools.partial(
   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:
+  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']
+  # Tez for Hive interactive related
+  tez_interactive_config_dir = os.path.realpath("/etc/tez_hive2/conf")
+  tez_interactive_user = config['configurations']['tez-env']['tez_user']
+
 
 # ranger host
 ranger_admin_hosts = default("/clusterHostInfo/ranger_admin_hosts", [])

http://git-wip-us.apache.org/repos/asf/ambari/blob/d4ab6fdf/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/status_params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/status_params.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/status_params.py
index a6e4a47..d71e061 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/status_params.py
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/status_params.py
@@ -27,6 +27,7 @@ from resource_management.libraries.functions.default import default
 from resource_management.libraries.functions import get_kinit_path
 from resource_management.libraries.script.script import Script
 
+
 # a map of the Ambari role to the component name
 # for use with <stack-root>/current/<component>
 SERVER_ROLE_DIRECTORY_MAP = {
@@ -83,6 +84,7 @@ else:
 
   # !!! required by ranger to be at this location unless HDP 2.3+
   hive_server_conf_dir = "/etc/hive/conf.server"
+
   hive_server_interactive_conf_dir = "/etc/hive2/conf.server"
 
   # HDP 2.2+
@@ -103,7 +105,7 @@ else:
     hive_conf_dir = hive_server_conf_dir
 
   if Script.is_stack_greater_or_equal("2.5"):
-    hive_server_interactive_conf_dir = format("/usr/hdp/current/{component_directory_interactive}/conf/conf.server")
+    hive_server_interactive_conf_dir = format("/usr/hdp/current/{component_directory_interactive}/conf")
 
   hive_config_dir = hive_client_conf_dir
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/d4ab6fdf/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/templates/startHiveserver2Interactive.sh.j2
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/templates/startHiveserver2Interactive.sh.j2 b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/templates/startHiveserver2Interactive.sh.j2
new file mode 100644
index 0000000..6062a7e
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/templates/startHiveserver2Interactive.sh.j2
@@ -0,0 +1,24 @@
+#
+#
+# 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.
+#
+#
+
+HIVE_SERVER2_INTERACTIVE_OPTS=" -hiveconf hive.log.file=hiveserver2Interactive.log -hiveconf hive.log.dir=$5"
+HIVE_INTERACTIVE_CONF_DIR=$4 {{hive_interactive_bin}}/hiveserver2 -hiveconf hive.metastore.uris=" " ${HIVE_SERVER2_INTERACTIVE_OPTS} > $1 2> $2 &
+echo $!|cat>$3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/d4ab6fdf/ambari-server/src/main/resources/stacks/HDP/2.0.6/properties/tarball_map.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/properties/tarball_map.json b/ambari-server/src/main/resources/stacks/HDP/2.0.6/properties/tarball_map.json
index 1e349a1..b70af13 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/properties/tarball_map.json
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/properties/tarball_map.json
@@ -7,10 +7,18 @@
     "{{ stack_root }}/{{ stack_version }}/tez/lib/tez.tar.gz",
     "/{{ stack_name }}/apps/{{ stack_version }}/tez/tez.tar.gz"
   ],
+  "tez_hive2": [
+    "{{ stack_root }}/{{ stack_version }}/tez_hive2/lib/tez.tar.gz",
+    "/{{ stack_name }}/apps/{{ stack_version }}/tez_hive2/tez.tar.gz"
+  ],
   "hive": [
     "{{ stack_root }}/{{ stack_version }}/hive/hive.tar.gz",
     "/{{ stack_name }}/apps/{{ stack_version }}/hive/hive.tar.gz"
   ],
+  "hive2": [
+    "{{ stack_root }}/{{ stack_version }}/hive/hive.tar.gz",
+    "/{{ stack_name }}/apps/{{ stack_version }}/hive2/hive.tar.gz"
+  ],
   "pig": [
     "{{ stack_root }}/{{ stack_version }}/pig/pig.tar.gz",
     "/{{ stack_name }}/apps/{{ stack_version }}/pig/pig.tar.gz"