You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by sw...@apache.org on 2016/04/20 22:58:30 UTC

ambari git commit: AMBARI-15985. Service Checks for 'Hive Server Interactive' and 'LLAP'.

Repository: ambari
Updated Branches:
  refs/heads/trunk 7046b2ba1 -> e6a449f5e


AMBARI-15985. Service Checks for 'Hive Server Interactive' and 'LLAP'.


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

Branch: refs/heads/trunk
Commit: e6a449f5e09826604cfb25ddce6c034338c4fb9f
Parents: 7046b2b
Author: Swapan Shridhar <ss...@hortonworks.com>
Authored: Tue Apr 19 23:59:56 2016 -0700
Committer: Swapan Shridhar <ss...@hortonworks.com>
Committed: Wed Apr 20 13:58:25 2016 -0700

----------------------------------------------------------------------
 .../0.12.0.2.0/package/files/hiveLlapSmoke.sh   |  40 +++++++
 .../0.12.0.2.0/package/scripts/params_linux.py  |   6 +
 .../0.12.0.2.0/package/scripts/service_check.py | 113 ++++++++++++++-----
 3 files changed, 133 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/e6a449f5/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/files/hiveLlapSmoke.sh
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/files/hiveLlapSmoke.sh b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/files/hiveLlapSmoke.sh
new file mode 100644
index 0000000..ec5b064
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/files/hiveLlapSmoke.sh
@@ -0,0 +1,40 @@
+#!/usr/bin/env bash
+#
+#
+# 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.
+#
+#
+
+: '
+ Parameters:
+ $1 : Stack root
+ $2 : Name of table to be created
+ $3 : Command name.
+'
+export tableName=$2
+
+case "$3" in
+
+prepare)
+  $1/current/hive-server2-hive2/bin/hive --hiveconf "hiveLlapServiceCheck=$tableName" -f $1/current/hive-server2-hive2/scripts/llap/sql/serviceCheckScript.sql
+;;
+
+cleanup)
+;;
+
+esac
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/e6a449f5/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 2414e8b..122756a 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
@@ -506,6 +506,12 @@ if has_hive_interactive:
   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)
+  # 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']

http://git-wip-us.apache.org/repos/asf/ambari/blob/e6a449f5/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/service_check.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/service_check.py
index 251e71f..f7304f7 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/service_check.py
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/service_check.py
@@ -22,10 +22,15 @@ 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):
@@ -47,60 +52,116 @@ class HiveServiceCheckWindows(HiveServiceCheck):
 
 @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)
 
-    address_list = params.hive_server_hosts
-
-    if not address_list:
-      raise Fail("Can not find any Hive Server host. Please check configuration.")
-
-    port = int(format("{hive_server_port}"))
-    print "Test connectivity to hive server"
     if params.security_enabled:
-      kinitcmd=format("{kinit_path_local} -kt {smoke_user_keytab} {smokeuser_principal}; ")
+      kinit_cmd = format(
+        "{kinit_path_local} -kt {smoke_user_keytab} {smokeuser_principal}; ")
     else:
-      kinitcmd=None
+      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)
+
+
+    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
 
-    print "Waiting for the Hive server to start..."
-      
+    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, port, params.hive_server2_authentication,
-                               params.hive_server_principal, kinitcmd, params.smokeuser,
+        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)
-        print "Successfully connected to %s on port %s" % (address, port)
+        Logger.info("Successfully connected to {0} on port {1}".format(address, server_port))
         workable_server_available = True
       except:
-        print "Connection to %s on port %s failed" % (address, port)
+        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 Hive server %s on port %s failed after %d seconds" %
-                 (params.hostname, params.hive_server_port, elapsed_time))
-    
-    print "Successfully connected to Hive at %s on port %s after %d seconds" %\
-          (params.hostname, params.hive_server_port, elapsed_time)
+      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))
 
-    hcat_service_check()
-    webhcat_service_check()
+    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))
+
+  def check_llap(self, env, kinit_cmd):
+    import params
+    env.set_params(params)
+
+    File(format("{tmp_dir}/hiveLlapSmoke.sh"),
+         content=StaticFile("hiveLlapSmoke.sh"),
+         mode=0755
+         )
+    unique_id = get_unique_id_and_date()
+    llap_cmd = format("{kinit_cmd}env JAVA_HOME={java64_home} {tmp_dir}/hiveLlapSmoke.sh {stack_root} llap_smoke_{unique_id} prepare")
+
+    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
+
+    Execute(llap_cmd,
+            user=params.hive_user,
+            path=['/usr/sbin', '/usr/local/bin', '/bin', '/usr/bin', exec_path],
+            tries=1,
+            try_sleep=5,
+            wait_for_finish=True,
+            stderr=subprocess.PIPE,
+            logoutput=True)
 
 if __name__ == "__main__":
-  HiveServiceCheck().execute()
+  HiveServiceCheck().execute()
\ No newline at end of file