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

ambari git commit: AMBARI-15723. Lifecycle for restarting LLAP on HiveServer Interactive via Custom Command (alejandro)

Repository: ambari
Updated Branches:
  refs/heads/trunk 1d93c53b8 -> 97c11244b


AMBARI-15723. Lifecycle for restarting LLAP on HiveServer Interactive via Custom Command (alejandro)


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

Branch: refs/heads/trunk
Commit: 97c11244b3f521618ef45683d815dfdf432dc573
Parents: 1d93c53
Author: Alejandro Fernandez <af...@hortonworks.com>
Authored: Mon Apr 4 20:09:48 2016 -0700
Committer: Alejandro Fernandez <af...@hortonworks.com>
Committed: Wed Apr 6 11:17:48 2016 -0700

----------------------------------------------------------------------
 .../HIVE/0.12.0.2.0/metainfo.xml                |   1 +
 .../package/scripts/hive_server_interactive.py  | 140 ++++++++++++++++---
 .../package/scripts/hive_service_interactive.py |  90 +-----------
 .../0.12.0.2.0/package/scripts/params_linux.py  |   4 +
 .../stacks/HDP/2.5/services/HIVE/metainfo.xml   |  10 ++
 ambari-web/app/controllers/main/service/item.js |  39 ++++++
 ambari-web/app/messages.js                      |   4 +
 ambari-web/app/models/host_component.js         |   7 +
 ambari-web/app/views/main/service/item.js       |  23 ++-
 ambari-web/test/views/main/service/item_test.js |   1 +
 10 files changed, 209 insertions(+), 110 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/97c11244/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/metainfo.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/metainfo.xml b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/metainfo.xml
index 4842f32..1f09364 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/metainfo.xml
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/metainfo.xml
@@ -330,6 +330,7 @@
         <config-type>hive-log4j</config-type>
         <config-type>hive-exec-log4j</config-type>
         <config-type>hive-env</config-type>
+        <!-- TODO, hive-interactive-site shouldn't be in common-services. -->
         <config-type>hive-interactive-site</config-type>
         <config-type>webhcat-site</config-type>
         <config-type>webhcat-env</config-type>

http://git-wip-us.apache.org/repos/asf/ambari/blob/97c11244/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 e1c0c42..8e4e6df 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
@@ -17,31 +17,30 @@ See the License for the specific language governing permissions and
 limitations under the License.
 
 """
-
-
+# Python Imports
+import subprocess
+import os
+import re
+import time
+import shutil
+
+# Ambari Commons & Resource Management imports
 from resource_management.libraries.script.script import Script
-from resource_management.libraries.resources.hdfs_resource import HdfsResource
-from resource_management.libraries.functions import conf_select
-from resource_management.libraries.functions import stack_select
 from resource_management.libraries.functions import format
-from resource_management.libraries.functions.copy_tarball import copy_to_hdfs
-from resource_management.libraries.functions.get_stack_version import get_stack_version
 from resource_management.libraries.functions.check_process_status import check_process_status
-from resource_management.libraries.functions.version import compare_versions, format_stack_version
-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.resources.system import Execute
 from resource_management.core import shell
-from subprocess import call
+from resource_management.core.exceptions import Fail
+from resource_management.core.logger import Logger
 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
-from hive_service_interactive import hive_service_interactive, stop_llap, start_llap
+
+# Local Imports
+from setup_ranger_hive import setup_ranger_hive
+from hive_service_interactive import hive_service_interactive
 from hive_interactive import hive_interactive
 
+
 class HiveServerInteractive(Script):
     def install(self, env):
       import params
@@ -77,8 +76,9 @@ class HiveServerDefault(HiveServerInteractive):
       # 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)
+      # Start LLAP before Hive Server Interactive start.
+      # TODO, why does LLAP have to be started before Hive Server Interactive???
+      status = self._llap_start(env)
       if status:
         # TODO : test the workability of Ranger and Hive2 during upgrade
         # setup_ranger_hive(upgrade_type=upgrade_type)
@@ -91,11 +91,13 @@ class HiveServerDefault(HiveServerInteractive):
       import params
       env.set_params(params)
 
+      # TODO, why must Hive Server Interactive be stopped before LLAP???
+
       # Stop Hive Interactive Server first
       # TODO : Upgrade check comes here.
-      hive_service_interactive('hiveserver2', action = 'stop')
+      hive_service_interactive('hiveserver2', action='stop')
 
-      stop_llap(self)
+      self._llap_stop(env)
 
     def status(self, env):
       import status_params
@@ -113,6 +115,102 @@ class HiveServerDefault(HiveServerInteractive):
     def security_status(self, env):
       pass
 
+    def restart_llap(self, env):
+      """
+      Custom command to Restart LLAP
+      """
+      Logger.info("Custom Command to retart LLAP")
+      import params
+      env.set_params(params)
+      self._llap_stop(env)
+      self._llap_start(env)
+
+    def _llap_stop(self, env):
+      import params
+      Logger.info("Stopping LLAP")
+      SLIDER_APP_NAME = "llap0"
+
+      stop_cmd = ["slider", "stop", SLIDER_APP_NAME]
+      Logger.info(format("Command: {stop_cmd}"))
+
+      code, output, error = shell.call(stop_cmd, user=params.hive_user, stderr=subprocess.PIPE, logoutput=True)
+      if code == 0:
+        Logger.info(format("Stopped {SLIDER_APP_NAME} application on Slider successfully"))
+      elif code == 69 and output is not None and "Unknown application instance" in output:
+        Logger.info(format("Application {SLIDER_APP_NAME} was already stopped on Slider"))
+      else:
+        raise Fail(format("Could not stop application {SLIDER_APP_NAME} on Slider"))
+
+      # Will exit with code 4 if need to run with "--force" to delete directories and registries.
+      destroy_cmd = ['slider', 'destroy', SLIDER_APP_NAME, "--force"]
+      code, output, error = shell.call(destroy_cmd, user=params.hive_user, stderr=subprocess.PIPE)
+      if code == 0:
+        Logger.info(format("Successfully removed slider app {SLIDER_APP_NAME}."))
+      else:
+        message = format("Could not remove slider app {SLIDER_APP_NAME}.")
+        if error is not None:
+          message += " " + error
+        raise Fail(message)
+
+    def _llap_start(self, env, cleanup=False):
+      import params
+      env.set_params(params)
+      Logger.info("Starting LLAP")
+
+      # TODO, start only if not already running.
+
+      # TODO, remove this prop after Hive is parametrized
+      stack_home = "/usr/hdp"
+
+      # TODO : Currently hardcoded the params. Need to read the suggested values from hive2/hive-site.xml.
+      # TODO, ensure that script works as hive from cmd when not cd'ed in /homve/hive
+      # Needs permission to write to hive home dir.
+      cmd = format("{stack_home}/current/hive-server2-hive2/bin/hive --service llap --instances 1 -slider-am-container-mb {slider_am_container_mb} --loglevel INFO")
+
+      run_file_path = None
+      try:
+        Logger.info(format("Command: {cmd}"))
+        cmd = cmd.split()
+        code, output, error = shell.checked_call(cmd, user=params.hive_user, stderr=subprocess.PIPE, logoutput=True)
+
+        if code != 0 or output is None:
+          raise Fail("Command failed with either non-zero return code or no output.")
+
+        # E.g., output:
+        # Prepared llap-slider-05Apr2016/run.sh for running LLAP on Slider
+        exp = r"Prepared (.*?run.sh) for running LLAP"
+        m = re.match(exp, output, re.I)
+        if m and len(m.groups()) == 1:
+          run_file_name = m.group(1)
+          run_file_path = os.path.join(params.hive_user_home_dir, run_file_name)
+        else:
+          raise Fail("Did not find run.sh file in output: " + str(output))
+
+        Logger.info(format("Run file path: {run_file_path}"))
+
+        if os.path.isfile(run_file_path):
+          Execute(run_file_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.
+          Logger.info("Sleeping for 30 secs")
+          time.sleep(30)
+          Logger.info("LLAP app deployed successfully.")
+        else:
+          raise Fail(format("Did not find run file {run_file_path}"))
+      except:
+        # Attempt to clean up the packaged application, or potentially rename it with a .bak
+        if run_file_path is not None and cleanup:
+          try:
+            parent_dir = os.path.dirname(run_file_path)
+            if os.path.isdir(parent_dir):
+              shutil.rmtree(parent_dir)
+          except Exception, e:
+            Logger.error("Could not cleanup LLAP app package. Error: " + str(e))
+
+        # throw the original exception
+        raise
+
 
 if __name__ == "__main__":
     HiveServerInteractive().execute()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/97c11244/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
index 9f32d16..92e67c7 100644
--- 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
@@ -18,27 +18,18 @@ limitations under the License.
 
 """
 
-import os
-import time
-import sys
-import shutil
-import subprocess
+# Python Imports
 
-from ambari_commons.constants import UPGRADE_TYPE_ROLLING
+# Ambari Commons & Resource Management imports
 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
 
+# Local Imports
+from hive_service import check_fs_root
 
 
 @OsFamilyFuncImpl(os_family=OSConst.WINSRV_FAMILY)
@@ -101,75 +92,4 @@ def hive_service_interactive(name, action='start', upgrade_type=None):
 
     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()))
+         )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/97c11244/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 70b8921..f1560ab 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
@@ -305,6 +305,10 @@ start_hiveserver2_script = 'startHiveserver2.sh.j2'
 start_metastore_script = 'startMetastore.sh'
 hive_metastore_pid = status_params.hive_metastore_pid
 
+# Hive Server Interactive
+# TODO , not being passed right now
+slider_am_container_mb = default("/configurations/hive-site-interactive/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']

http://git-wip-us.apache.org/repos/asf/ambari/blob/97c11244/ambari-server/src/main/resources/stacks/HDP/2.5/services/HIVE/metainfo.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.5/services/HIVE/metainfo.xml b/ambari-server/src/main/resources/stacks/HDP/2.5/services/HIVE/metainfo.xml
index b434021..85f0eb6 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.5/services/HIVE/metainfo.xml
+++ b/ambari-server/src/main/resources/stacks/HDP/2.5/services/HIVE/metainfo.xml
@@ -91,6 +91,16 @@
             <script>scripts/hive_server_interactive.py</script>
             <scriptType>PYTHON</scriptType>
           </commandScript>
+          <customCommands>
+            <customCommand>
+              <name>RESTART_LLAP</name>
+              <commandScript>
+                <script>scripts/hive_server_interactive.py</script>
+                <scriptType>PYTHON</scriptType>
+                <timeout>600</timeout>
+              </commandScript>
+            </customCommand>
+          </customCommands>
           <configuration-dependencies>
             <config-type>hive-site</config-type>
             <config-type>hive-interactive-site</config-type>

http://git-wip-us.apache.org/repos/asf/ambari/blob/97c11244/ambari-web/app/controllers/main/service/item.js
----------------------------------------------------------------------
diff --git a/ambari-web/app/controllers/main/service/item.js b/ambari-web/app/controllers/main/service/item.js
index ef728d1..46488d9 100644
--- a/ambari-web/app/controllers/main/service/item.js
+++ b/ambari-web/app/controllers/main/service/item.js
@@ -599,6 +599,45 @@ App.MainServiceItemController = Em.Controller.extend(App.SupportClientConfigsDow
     App.showAlertPopup(Em.I18n.t('services.service.actions.run.yarnRefreshQueues.error'), error);
   },
 
+  restartLLAP: function(event) {
+    var context = Em.I18n.t('services.service.actions.run.restartLLAP');
+    this.manageLLAP('RESTART_LLAP', context);
+  },
+  manageLLAP: function(command, context) {
+    var controller = this;
+    var host = App.HostComponent.find().findProperty('componentName', 'HIVE_SERVER_INTERACTIVE').get('hostName');
+    return App.showConfirmationPopup(function() {
+      App.ajax.send({
+        name: 'service.item.executeCustomCommand',
+        sender: controller,
+        data: {
+          command: command,
+          context: context,
+          hosts: host,
+          serviceName: "HIVE",
+          componentName: "HIVE_SERVER_INTERACTIVE"
+        },
+        success: 'manageLLAPSuccessCallback',
+        error: 'manageLLAPErrorCallback'
+      });
+    });
+  },
+  manageLLAPSuccessCallback : function(data, ajaxOptions, params) {
+    if (data.Requests.id) {
+      App.router.get('backgroundOperationsController').showPopup();
+    }
+  },
+  manageLLAPErrorCallback : function(data) {
+    var error = Em.I18n.t('services.service.actions.run.executeCustomCommand.error');
+    if (data && data.responseText) {
+      try {
+        var json = $.parseJSON(data.responseText);
+        error += json.message;
+      } catch (err) {}
+    }
+    App.showAlertPopup(Em.I18n.t('common.error'), error);
+  },
+
   /**
    * On click handler for rebalance Hdfs command from items menu
    */

http://git-wip-us.apache.org/repos/asf/ambari/blob/97c11244/ambari-web/app/messages.js
----------------------------------------------------------------------
diff --git a/ambari-web/app/messages.js b/ambari-web/app/messages.js
index 4cb02d9..b2e3f1c 100644
--- a/ambari-web/app/messages.js
+++ b/ambari-web/app/messages.js
@@ -1752,6 +1752,10 @@ Em.I18n.translations = {
   'services.service.actions.run.stopLdapKnox.title':'Stop Demo LDAP Knox Gateway',
   'services.service.actions.run.stopLdapKnox.context':'Stop Demo LDAP',
   'services.service.actions.run.startStopLdapKnox.error': 'Error during remote command: ',
+
+  // LLAP Custom Command
+  'services.service.actions.run.restartLLAP':'Restart LLAP',
+
   'services.service.actions.run.immediateStopHawqService.context':'Stop HAWQ Service (Immediate Mode)',
   'services.service.actions.run.immediateStopHawqService.label':'Stop HAWQ Service (Immediate Mode)',
   'services.service.actions.run.immediateStopHawqSegment.label':'Stop (Immediate Mode)',

http://git-wip-us.apache.org/repos/asf/ambari/blob/97c11244/ambari-web/app/models/host_component.js
----------------------------------------------------------------------
diff --git a/ambari-web/app/models/host_component.js b/ambari-web/app/models/host_component.js
index d7dce6e..9edc76f 100644
--- a/ambari-web/app/models/host_component.js
+++ b/ambari-web/app/models/host_component.js
@@ -362,6 +362,13 @@ App.HostComponentActionMap = {
         cssClass: 'icon-stop',
         disabled: false
       },
+      RESTART_LLAP: {
+        action: 'restartLLAP',
+        customCommand: 'RESTART_LLAP',
+        context: Em.I18n.t('services.service.actions.run.restartLLAP'),
+        label: Em.I18n.t('services.service.actions.run.restartLLAP') + ' ∞',
+        cssClass: 'icon-refresh'
+      },
       REBALANCEHDFS: {
         action: 'rebalanceHdfsNodes',
         customCommand: 'REBALANCEHDFS',

http://git-wip-us.apache.org/repos/asf/ambari/blob/97c11244/ambari-web/app/views/main/service/item.js
----------------------------------------------------------------------
diff --git a/ambari-web/app/views/main/service/item.js b/ambari-web/app/views/main/service/item.js
index 6fc5068..790c0dc 100644
--- a/ambari-web/app/views/main/service/item.js
+++ b/ambari-web/app/views/main/service/item.js
@@ -28,7 +28,9 @@ App.MainServiceItemView = Em.View.extend({
   isPassive: Em.computed.equal('controller.content.passiveState', 'ON'),
 
   /**
-   * Some custom commands need custom logic to be executed
+   * Some custom commands need custom logic to be execute.
+   * Typically, these services already have Custom Commands, so we must exclude the default ones
+   * in order to make more changes to them like icons and rules.
    */
   mastersExcludedCommands: {
     'NAMENODE': ['DECOMMISSION', 'REBALANCEHDFS'],
@@ -37,7 +39,8 @@ App.MainServiceItemView = Em.View.extend({
     'KNOX_GATEWAY': ['STARTDEMOLDAP','STOPDEMOLDAP'],
     'HAWQMASTER': ['IMMEDIATE_STOP_HAWQ_SERVICE', 'RUN_HAWQ_CHECK', 'HAWQ_CLEAR_CACHE', 'REMOVE_HAWQ_STANDBY'],
     'HAWQSEGMENT': ['IMMEDIATE_STOP_HAWQ_SEGMENT'],
-    'HAWQSTANDBY' : ['RESYNC_HAWQ_STANDBY','ACTIVATE_HAWQ_STANDBY']
+    'HAWQSTANDBY': ['RESYNC_HAWQ_STANDBY','ACTIVATE_HAWQ_STANDBY'],
+    'HIVE_SERVER_INTERACTIVE' : ["RESTART_LLAP"],
   },
 
    addActionMap: function() {
@@ -178,8 +181,8 @@ App.MainServiceItemView = Em.View.extend({
       }
       options.push(actionMap.TOGGLE_PASSIVE);
       var serviceName = service.get('serviceName');
-      var nnComponent = App.StackServiceComponent.find().findProperty('componentName','NAMENODE');
-      var knoxGatewayComponent = App.StackServiceComponent.find().findProperty('componentName','KNOX_GATEWAY');
+      var nnComponent = App.StackServiceComponent.find().findProperty('componentName', 'NAMENODE');
+      var knoxGatewayComponent = App.StackServiceComponent.find().findProperty('componentName', 'KNOX_GATEWAY');
       if (serviceName === 'HDFS' && nnComponent) {
         var namenodeCustomCommands = nnComponent.get('customCommands');
         if (namenodeCustomCommands && namenodeCustomCommands.contains('REBALANCEHDFS')) {
@@ -196,6 +199,18 @@ App.MainServiceItemView = Em.View.extend({
         });
       }
 
+      if (serviceName === 'HIVE') {
+        var hiveServerInteractiveComponent = App.StackServiceComponent.find().findProperty('componentName', 'HIVE_SERVER_INTERACTIVE');
+        if (hiveServerInteractiveComponent) {
+          var LLAPCustomCommands = hiveServerInteractiveComponent.get('customCommands');
+          LLAPCustomCommands.forEach(function (command) {
+            if (actionMap[command]) {
+              options.push(actionMap[command]);
+            }
+          });
+        }
+      }
+
       /**
        * Display all custom commands of Master and StandBy on Service page.
        **/

http://git-wip-us.apache.org/repos/asf/ambari/blob/97c11244/ambari-web/test/views/main/service/item_test.js
----------------------------------------------------------------------
diff --git a/ambari-web/test/views/main/service/item_test.js b/ambari-web/test/views/main/service/item_test.js
index 9feb187..3c0e6aa 100644
--- a/ambari-web/test/views/main/service/item_test.js
+++ b/ambari-web/test/views/main/service/item_test.js
@@ -504,6 +504,7 @@ describe('App.MainServiceItemView', function () {
           case 'APP_TIMELINE_SERVER':
           case 'KAFKA_BROKER':
           case 'OOZIE_SERVER':
+          case 'HIVE_SERVER_INTERACTIVE':
           case 'NIMBUS':
             return Em.Object.create({ customCommands: [] });
           default: