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

ambari git commit: AMBARI-16085. Modify Ambari stacks for Ranger (for enabling plugins) to use service keytab for creating repositories and policies (gautam)

Repository: ambari
Updated Branches:
  refs/heads/trunk 237a5d692 -> e74bfa4a4


AMBARI-16085. Modify Ambari stacks for Ranger (for enabling plugins) to use service keytab for creating repositories and policies (gautam)


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

Branch: refs/heads/trunk
Commit: e74bfa4a4ba7689f7f7d3a363fc817871b32c798
Parents: 237a5d6
Author: Gautam Borad <ga...@apache.org>
Authored: Wed Apr 27 14:38:48 2016 +0530
Committer: Gautam Borad <ga...@apache.org>
Committed: Thu Apr 28 10:27:21 2016 +0530

----------------------------------------------------------------------
 .../libraries/functions/curl_krb_request.py     |  21 +-
 .../libraries/functions/ranger_functions_v2.py  | 278 +++++++++++++++++--
 .../functions/setup_ranger_plugin_xml.py        |  22 +-
 .../0.96.0.2.0/package/scripts/params_linux.py  |  22 ++
 .../package/scripts/setup_ranger_hbase.py       |  39 ++-
 .../2.1.0.2.0/package/scripts/params_linux.py   |  13 +
 .../package/scripts/setup_ranger_hdfs.py        |  62 +++--
 .../0.12.0.2.0/package/scripts/params_linux.py  |  15 +
 .../package/scripts/setup_ranger_hive.py        |  35 ++-
 .../KAFKA/0.8.1.2.2/package/scripts/params.py   |   7 +
 .../package/scripts/setup_ranger_kafka.py       |   8 +-
 .../0.5.0.2.2/package/scripts/params_linux.py   |  17 +-
 .../package/scripts/setup_ranger_knox.py        |  38 ++-
 .../0.4.0/package/scripts/setup_ranger_xml.py   |  27 +-
 .../0.9.1.2.1/package/scripts/params_linux.py   |  20 ++
 .../package/scripts/setup_ranger_storm.py       |  35 ++-
 .../2.1.0.2.0/package/scripts/params_linux.py   |  14 +-
 .../package/scripts/setup_ranger_yarn.py        |  10 +-
 18 files changed, 577 insertions(+), 106 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-common/src/main/python/resource_management/libraries/functions/curl_krb_request.py
----------------------------------------------------------------------
diff --git a/ambari-common/src/main/python/resource_management/libraries/functions/curl_krb_request.py b/ambari-common/src/main/python/resource_management/libraries/functions/curl_krb_request.py
index cf0d5a6..2acf871 100644
--- a/ambari-common/src/main/python/resource_management/libraries/functions/curl_krb_request.py
+++ b/ambari-common/src/main/python/resource_management/libraries/functions/curl_krb_request.py
@@ -62,7 +62,7 @@ KERBEROS_KINIT_TIMER_PARAMETER = "kerberos.kinit.timer"
 def curl_krb_request(tmp_dir, keytab, principal, url, cache_file_prefix,
     krb_exec_search_paths, return_only_http_code, caller_label, user,
     connection_timeout = CONNECTION_TIMEOUT_DEFAULT,
-    kinit_timer_ms=DEFAULT_KERBEROS_KINIT_TIMER_MS):
+    kinit_timer_ms=DEFAULT_KERBEROS_KINIT_TIMER_MS, method = '',body='',header=''):
   """
   Makes a curl request using the kerberos credentials stored in a calculated cache file. The
   cache file is created by combining the supplied principal, keytab, user, and request name into
@@ -180,10 +180,23 @@ def curl_krb_request(tmp_dir, keytab, principal, url, cache_file_prefix,
                              '%{http_code}', url, '--connect-timeout', str(connection_timeout), '--max-time', str(maximum_timeout), '-o', '/dev/null'],
                              user=user, env=kerberos_env)
     else:
+      curl_command = ['curl', '-L', '-k', '--negotiate', '-u', ':', '-b', cookie_file, '-c', cookie_file,
+                      url, '--connect-timeout', str(connection_timeout), '--max-time', str(maximum_timeout)]
       # returns response body
-      _, curl_stdout, curl_stderr = get_user_call_output(['curl', '-L', '-k', '--negotiate', '-u', ':', '-b', cookie_file, '-c', cookie_file,
-                             url, '--connect-timeout', str(connection_timeout), '--max-time', str(maximum_timeout)],
-                             user=user, env=kerberos_env)
+      if len(method) > 0 and len(body) == 0 and len(header) == 0:
+        curl_command.extend(['-X', method])
+
+      elif len(method) > 0 and len(body) == 0 and len(header) > 0:
+        curl_command.extend(['-H', header, '-X', method])
+
+      elif len(method) > 0 and len(body) > 0 and len(header) == 0:
+        curl_command.extend(['-X', method, '-d', body])
+
+      elif len(method) > 0 and len(body) > 0 and len(header) > 0:
+        curl_command.extend(['-H', header, '-X', method, '-d', body])
+
+      _, curl_stdout, curl_stderr = get_user_call_output(curl_command, user=user, env=kerberos_env)
+
   except Fail:
     if logger.isEnabledFor(logging.DEBUG):
       logger.exception("Unable to make a curl request for {0}.".format(caller_label))

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-common/src/main/python/resource_management/libraries/functions/ranger_functions_v2.py
----------------------------------------------------------------------
diff --git a/ambari-common/src/main/python/resource_management/libraries/functions/ranger_functions_v2.py b/ambari-common/src/main/python/resource_management/libraries/functions/ranger_functions_v2.py
index 3bb0103..f081505 100644
--- a/ambari-common/src/main/python/resource_management/libraries/functions/ranger_functions_v2.py
+++ b/ambari-common/src/main/python/resource_management/libraries/functions/ranger_functions_v2.py
@@ -35,6 +35,8 @@ from ambari_commons.exceptions import TimeoutError
 from resource_management.core.exceptions import Fail
 from resource_management.libraries.functions.decorator import safe_retry
 from resource_management.libraries.functions.format import format
+from resource_management.libraries.functions.curl_krb_request import curl_krb_request
+from resource_management.core.environment import Environment
 
 
 class RangeradminV2:
@@ -47,6 +49,7 @@ class RangeradminV2:
     self.url_repos = self.base_url + '/service/assets/assets'
     self.url_repos_pub = self.base_url + '/service/public/v2/api/service'
     self.url_policies = self.base_url + '/service/public/v2/api/policy'
+    self.url_policies_get = self.base_url + '/service/public/v2/api/service/{servicename}/policy'
     self.url_groups = self.base_url + '/service/xusers/groups'
     self.url_users = self.base_url + '/service/xusers/users'
     self.url_sec_users = self.base_url + '/service/xusers/secure/users'
@@ -61,7 +64,7 @@ class RangeradminV2:
     :param name: name of the component, from which, function will search in list of repositories
     :param component:, component for which repository has to be checked
     :param status: active or inactive
-    :param usernamepassword: user credentials using which repository needs to be searched. 
+    :param usernamepassword: user credentials using which repository needs to be searched.
     :return: Returns Ranger repository object if found otherwise None
     """
     try:
@@ -91,50 +94,76 @@ class RangeradminV2:
       raise Fail("Ranger Admin service is not reachable, please restart the service and then try again")
     except TimeoutError:
       raise Fail("Connection to Ranger Admin failed. Reason - timeout")
-      
-    
-  def create_ranger_repository(self, component, repo_name, repo_properties, 
+
+
+  def create_ranger_repository(self, component, repo_name, repo_properties,
                                ambari_ranger_admin, ambari_ranger_password,
-                               admin_uname, admin_password, policy_user):
-    response_code = self.check_ranger_login_urllib2(self.base_url)
-    repo_data = json.dumps(repo_properties)
-    ambari_ranger_password = unicode(ambari_ranger_password)
-    admin_password = unicode(admin_password)
-    ambari_username_password_for_ranger = format('{ambari_ranger_admin}:{ambari_ranger_password}')
-
-    
-    if response_code is not None and response_code == 200:
-      user_resp_code = self.create_ambari_admin_user(ambari_ranger_admin, ambari_ranger_password, format("{admin_uname}:{admin_password}"))
-      if user_resp_code is not None and user_resp_code == 200:
+                               admin_uname, admin_password, policy_user, is_security_enabled, component_user, component_user_principal, component_user_keytab):
+    if not is_security_enabled :
+      response_code = self.check_ranger_login_urllib2(self.base_url)
+      repo_data = json.dumps(repo_properties)
+      ambari_ranger_password = unicode(ambari_ranger_password)
+      admin_password = unicode(admin_password)
+      ambari_username_password_for_ranger = format('{ambari_ranger_admin}:{ambari_ranger_password}')
+
+
+      if response_code is not None and response_code == 200:
+        user_resp_code = self.create_ambari_admin_user(ambari_ranger_admin, ambari_ranger_password, format("{admin_uname}:{admin_password}"))
+        if user_resp_code is not None and user_resp_code == 200:
+          retryCount = 0
+          while retryCount <= 5:
+            repo = self.get_repository_by_name_urllib2(repo_name, component, 'true', ambari_username_password_for_ranger)
+            if repo is not None:
+              Logger.info('{0} Repository {1} exist'.format(component.title(), repo['name']))
+              break
+            else:
+              response = self.create_repository_urllib2(repo_data, ambari_username_password_for_ranger)
+              if response is not None:
+                Logger.info('{0} Repository created in Ranger admin'.format(component.title()))
+                break
+              else:
+                if retryCount < 5:
+                  Logger.info("Retry Repository Creation is being called")
+                  time.sleep(30) # delay for 30 seconds
+                  retryCount += 1
+                else:
+                  Logger.error('{0} Repository creation failed in Ranger admin'.format(component.title()))
+                  break
+        else:
+          Logger.error('Ambari admin user creation failed')
+      elif not self.skip_if_rangeradmin_down:
+        Logger.error("Connection failed to Ranger Admin !")
+    else:
+      response = self.check_ranger_login_curl(component_user,component_user_keytab,component_user_principal,self.base_url,True)
+
+      if response and response[0] == 200:
         retryCount = 0
+        repo_data = json.dumps(repo_properties)
         while retryCount <= 5:
-          repo = self.get_repository_by_name_urllib2(repo_name, component, 'true', ambari_username_password_for_ranger)
-          if repo is not None:
-            Logger.info('{0} Repository {1} exist'.format(component.title(), repo['name']))
+          response = self.get_repository_by_name_curl(component_user,component_user_keytab,component_user_principal,repo_name, component, 'true')
+          if response is not None:
+            Logger.info('{0} Repository {1} exist'.format(component.title(), (response['name'])))
             break
           else:
-            response = self.create_repository_urllib2(repo_data, ambari_username_password_for_ranger)
-            if response is not None:
+            response = self.create_repository_curl(component_user,component_user_keytab,component_user_principal,repo_name, repo_data,policy_user)
+            if response and len(response) > 0:
               Logger.info('{0} Repository created in Ranger admin'.format(component.title()))
               break
             else:
               if retryCount < 5:
-                Logger.info("Retry Repository Creation is being called")
                 time.sleep(30) # delay for 30 seconds
                 retryCount += 1
               else:
                 Logger.error('{0} Repository creation failed in Ranger admin'.format(component.title()))
                 break
       else:
-        Logger.error('Ambari admin user creation failed')
-    elif not self.skip_if_rangeradmin_down:
-      Logger.error("Connection failed to Ranger Admin !")
+        Logger.error("Connection failed to Ranger Admin !")
 
   @safe_retry(times=5, sleep_time=8, backoff_factor=1.5, err_class=Fail, return_on_fail=None)
   def create_repository_urllib2(self, data, usernamepassword):
     """
     :param data: json object to create repository
-    :param usernamepassword: user credentials using which repository needs to be searched. 
+    :param usernamepassword: user credentials using which repository needs to be searched.
     :return: Returns created Ranger repository object
     """
     try:
@@ -169,8 +198,8 @@ class RangeradminV2:
   def check_ranger_login_urllib2(self, url):
     """
     :param url: ranger admin host url
-    :param usernamepassword: user credentials using which repository needs to be searched. 
-    :return: Returns login check response 
+    :param usernamepassword: user credentials using which repository needs to be searched.
+    :return: Returns login check response
     """
     try:
       response = openurl(url, timeout=20)
@@ -189,7 +218,7 @@ class RangeradminV2:
   @safe_retry(times=5, sleep_time=8, backoff_factor=1.5, err_class=Fail, return_on_fail=None)
   def create_ambari_admin_user(self, ambari_admin_username, ambari_admin_password, usernamepassword):
     """
-    :param ambari_admin_username: username of user to be created 
+    :param ambari_admin_username: username of user to be created
     :param ambari_admin_password: user password of user to be created
     :param usernamepassword: user credentials using which repository needs to be searched.
     :return: Returns user credentials if user exist otherwise rerutns credentials of  created user.
@@ -257,3 +286,196 @@ class RangeradminV2:
       raise Fail("Ranger Admin service is not reachable, please restart the service and then try again")
     except TimeoutError:
       raise Fail("Connection to Ranger Admin failed. Reason - timeout")
+
+
+  def call_curl_request(self,user,keytab,principal, url, flag_http_response, request_method='GET',request_body='',header=''):
+    """
+    :param user: service user for which call is to be made
+    :param keytab: keytab of service user
+    :param principal: principal of service user
+    :param url: url with which call is to be made
+    :param flag_http_response: flag to get only response-code or response string
+    :param request_method: http method (GET / POST / PUT / DELETE)
+    :param request_body: data to be send along with the request
+    :param header: http header required for the call
+    :return: Returns the response error_msg , time_millis
+    """
+    response = None
+    error_msg = None
+    time_millis = 0
+    response, error_msg, time_millis = curl_krb_request(Environment.get_instance().tmp_dir, keytab, principal, url, 'ranger_admin_calls',
+                                                         None, flag_http_response, "Ranger-Admin API calls", user,kinit_timer_ms=0,method = request_method,body=request_body,header=header)
+
+    return response, error_msg, time_millis
+
+  @safe_retry(times=75, sleep_time=8, backoff_factor=1, err_class=Fail, return_on_fail=None)
+  def check_ranger_login_curl(self, component_user,component_user_keytab,component_user_principal,base_url,True):
+    """
+    :param url: ranger admin host url
+    :param usernamepassword: user credentials using which repository needs to be searched.
+    :return: Returns login check response
+    """
+    response = ''
+    error_msg = ''
+    time_millis = 0
+    try:
+      response,error_msg,time_millis = self.call_curl_request(component_user,component_user_keytab,component_user_principal,self.base_url,True)
+    except Fail,fail:
+      raise Fail(fail.args)
+
+    return response, error_msg,time_millis
+
+
+
+  @safe_retry(times=5, sleep_time=8, backoff_factor=1.5, err_class=Fail, return_on_fail=None)
+  def get_repository_by_name_curl(self, component_user,component_user_keytab,component_user_principal,name, component, status):
+    """
+    :param component_user: service user for which call is to be made
+    :param component_user_keytab: keytab of service user
+    :param component_user_principal: principal of service user
+    :param name: name of the component, te be searched
+    :param component:, component for which repository has to be checked
+    :param status: active or inactive
+    :param usernamepassword: user credentials using which repository needs to be searched.
+    :return: Returns Ranger repository object if found otherwise None
+    """
+    try:
+      search_repo_url = self.url_repos_pub + "?serviceName=" + name + "&serviceType=" + component + "&isEnabled=" + status
+      response,error_message,time_in_millis = self.call_curl_request(component_user,component_user_keytab,component_user_principal,search_repo_url,False,request_method='GET')
+      response_stripped = response[1:len(response) - 1]
+      if response_stripped and len(response_stripped) > 0:
+        response_json = json.loads(response_stripped)
+        if response_json['name'].lower() == name.lower():
+          return response_json
+        else:
+          return None
+    except Fail, fail:
+      raise Fail(str(fail))
+
+
+
+  @safe_retry(times=5, sleep_time=8, backoff_factor=1.5, err_class=Fail, return_on_fail=None)
+  def create_repository_curl(self,component_user,component_user_keytab,component_user_principal,name, data,policy_user):
+    """
+    :param component_user: service user for which call is to be made
+    :param component_user_keytab: keytab of service user
+    :param component_user_principal: principal of service user
+    :param name: name of the repository to be created
+    :param data: service definition of the repository
+    :return:
+    """
+    search_repo_url = self.url_repos_pub
+    header = 'Content-Type: application/json'
+    method = 'POST'
+
+    response,error_message,time_in_millis = self.call_curl_request(component_user,component_user_keytab,component_user_principal,search_repo_url,False,method,data,header)
+    if response and len(response) > 0:
+      response_json = json.loads(response)
+      if response_json['name'].lower() == name.lower():
+        Logger.info('Repository created Successfully')
+        service_name = response_json['name']
+        service_type = response_json['type']
+        if service_type in ['hdfs','hive','hbase','knox','storm']:
+          policy_list = self.get_policy_by_repo_name(component_user,component_user_keytab,component_user_principal,service_name,service_type,'true')
+          if policy_list is not None and len(policy_list) > 0:
+            policy_update_count = 0
+            for policy in policy_list:
+              updated_policy_object = self.get_policy_params(service_type,policy,policy_user=policy_user)
+              response,error_message,time_in_millis = self.update_ranger_policy(component_user,component_user_keytab,component_user_principal,updated_policy_object['id'],json.dumps(updated_policy_object))
+              if response and len(response) > 0:
+                policy_update_count += 1
+              else:
+                Logger.info("Policy updated failed")
+            if len(policy_list) == policy_update_count:
+              Logger.info("Ranger Repository created successfully and policies updated successfully providing ambari-qa user all permissions")
+              return response_json
+      else:
+        Logger.info('Repository creation failed')
+        return None
+    else:
+      Logger.info('Repository creation failed')
+      return None
+
+
+
+  @safe_retry(times=5, sleep_time=8, backoff_factor=1.5, err_class=Fail, return_on_fail=None)
+  def get_policy_by_repo_name(self, component_user,component_user_keytab,component_user_principal,name, component, status):
+    """
+    :param name: repository name
+    :param component: component name for which policy needs to be searched
+    :param status: true or false
+    :param usernamepassword: user credentials using which policy needs to be searched
+    :return Returns successful response else None
+    """
+    try:
+      # time.sleep(5)
+      search_policy_url = self.url_policies_get+ '?serviceType=' + component + '&isEnabled=' + status
+
+      search_policy_url = search_policy_url.format(servicename=name)
+      method = 'GET'
+      response,error_message,time_in_millis = self.call_curl_request(component_user,component_user_keytab,component_user_principal,search_policy_url,False,request_method=method)
+      if response and len(response) > 0:
+        response = json.loads(response)
+        return response
+      else:
+        return None
+    except Fail, fail:
+      raise Fail(str(fail))
+
+  @safe_retry(times=5, sleep_time=8, backoff_factor=1.5, err_class=Fail, return_on_fail=None)
+  def update_ranger_policy(self,component_user,component_user_keytab,component_user_principal, policyId, data):
+    """
+    :param policyId: policy id which needs to be updated
+    :param data: policy data that needs to be updated
+    :param usernamepassword: user credentials using which policy needs to be updated
+    :return Returns successful response and response code else None
+    """
+    try:
+      update_url = self.url_policies + '/' + str(policyId)
+      header = 'Content-Type: application/json'
+      method = 'PUT'
+
+      response,error_message,time_in_millis = self.call_curl_request(component_user,component_user_keytab,component_user_principal,update_url,False,method,data,header=header)
+      if response and len(response) > 0:
+        Logger.info('Policy updated Successfully')
+        
+        response_json = json.loads(response)
+        return response_json,error_message,time_in_millis
+      else:
+        Logger.error('Update Policy failed')
+        return None, None,None
+    except Fail, fail:
+      raise Fail(str(fail))
+
+  def get_policy_params(self, typeOfPolicy, policyObj, policy_user):
+    """
+    :param typeOfPolicy: component name for which policy has to be get
+    :param policyObj: policy dict
+    :param policy_user: policy user that needs to be updated
+    :returns Returns updated policy dict
+    """
+    typeOfPolicy = typeOfPolicy.lower()
+    policy_record = ''
+    if typeOfPolicy == "hdfs":
+      policy_record  = {'users': [policy_user], 'accesses': [{'isAllowed': True,'type': 'read' }, {'isAllowed': True,'type': 'write' },{'isAllowed': True,'type': 'execute' }],'delegateAdmin': True}
+    elif typeOfPolicy == "hive":
+      policy_record = {'users': [policy_user],
+                                   'accesses': [{'isAllowed': True,'type': 'select' }, {'isAllowed': True,'type': 'update' }, {'isAllowed': True,'type': 'create' },
+                                                {'isAllowed': True,'type': 'drop' }, {'isAllowed': True,'type': 'alter' }, {'isAllowed': True,'type': 'index' },
+                                                {'isAllowed': True,'type': 'lock' }, {'isAllowed': True,'type': 'all' }],'delegateAdmin':True }
+    elif typeOfPolicy == "hbase":
+      policy_record = {'users': [policy_user], 'accesses': [{'isAllowed': True,'type': 'read' }, {'isAllowed': True,'type': 'write' },
+                                                             {'isAllowed': True,'type': 'create' }],'delegateAdmin':True }
+    elif typeOfPolicy == "knox":
+      policy_record = {'users': [policy_user], 'accesses': [{'isAllowed': True,'type': 'allow' }],'delegateAdmin':True }
+    elif typeOfPolicy == "storm":
+      policy_record = {'users': [policy_user],
+                                   'accesses': [{'isAllowed': True,'type': 'submitTopology' }, {'isAllowed': True,'type': 'fileUpload' },{'isAllowed': True,'type': 'getNimbusConf' },
+                                                {'isAllowed': True,'type': 'getClusterInfo' },{'isAllowed': True,'type': 'fileDownload' } , {'isAllowed': True,'type': 'killTopology' },
+                                                {'isAllowed': True,'type': 'rebalance' }, {'isAllowed': True,'type': 'activate' }, {'isAllowed': True,'type': 'deactivate' },
+                                                {'isAllowed': True,'type': 'getTopologyConf' }, {'isAllowed': True,'type': 'getTopology' }, {'isAllowed': True,'type': 'getUserTopology' },
+                                                {'isAllowed': True,'type': 'getTopologyInfo' }, {'isAllowed': True,'type': 'uploadNewCredential' }],'delegateAdmin':True}
+
+    if policy_record != '':
+      policyObj['policyItems'].append(policy_record)
+    return policyObj

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-common/src/main/python/resource_management/libraries/functions/setup_ranger_plugin_xml.py
----------------------------------------------------------------------
diff --git a/ambari-common/src/main/python/resource_management/libraries/functions/setup_ranger_plugin_xml.py b/ambari-common/src/main/python/resource_management/libraries/functions/setup_ranger_plugin_xml.py
index a7faf3b..4a071ca 100644
--- a/ambari-common/src/main/python/resource_management/libraries/functions/setup_ranger_plugin_xml.py
+++ b/ambari-common/src/main/python/resource_management/libraries/functions/setup_ranger_plugin_xml.py
@@ -45,7 +45,9 @@ def setup_ranger_plugin(component_select_name, service_name,
                         plugin_policymgr_ssl_properties, plugin_policymgr_ssl_attributes,
                         component_list, audit_db_is_enabled, credential_file, 
                         xa_audit_db_password, ssl_truststore_password,
-                        ssl_keystore_password, api_version=None, stack_version_override = None, skip_if_rangeradmin_down = True):
+                        ssl_keystore_password, api_version=None, stack_version_override = None, skip_if_rangeradmin_down = True,
+                        is_security_enabled = False, is_stack_supports_ranger_kerberos = False,
+                        component_user_principal = None, component_user_keytab = None):
 
   if audit_db_is_enabled:
     File(component_downloaded_custom_connector,
@@ -65,15 +67,25 @@ def setup_ranger_plugin(component_select_name, service_name,
     stack_version = stack_version_override
 
   component_conf_dir = conf_dict
-  
+
   if plugin_enabled:
+    if api_version is not None and api_version == 'v2':
 
-    if api_version == 'v2' and api_version is not None:
       ranger_adm_obj = RangeradminV2(url=policymgr_mgr_url, skip_if_rangeradmin_down=skip_if_rangeradmin_down)
+      if is_security_enabled and is_stack_supports_ranger_kerberos:
+        ranger_adm_obj.create_ranger_repository(service_name, repo_name, plugin_repo_dict,
+                                              ranger_env_properties['ranger_admin_username'], ranger_env_properties['ranger_admin_password'],
+                                              ranger_env_properties['admin_username'], ranger_env_properties['admin_password'],
+                                              policy_user,is_security_enabled,component_user,component_user_principal,component_user_keytab)
+
+      else:
+        ranger_adm_obj.create_ranger_repository(service_name, repo_name, plugin_repo_dict,
+                                                ranger_env_properties['ranger_admin_username'], ranger_env_properties['ranger_admin_password'],
+                                                ranger_env_properties['admin_username'], ranger_env_properties['admin_password'],
+                                                policy_user)
     else:
       ranger_adm_obj = Rangeradmin(url=policymgr_mgr_url, skip_if_rangeradmin_down=skip_if_rangeradmin_down)
-
-    ranger_adm_obj.create_ranger_repository(service_name, repo_name, plugin_repo_dict,
+      ranger_adm_obj.create_ranger_repository(service_name, repo_name, plugin_repo_dict,
                                             ranger_env_properties['ranger_admin_username'], ranger_env_properties['ranger_admin_password'],
                                             ranger_env_properties['admin_username'], ranger_env_properties['admin_password'],
                                             policy_user)

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params_linux.py
index d8a89e7..18283c4 100644
--- a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params_linux.py
@@ -55,6 +55,7 @@ etc_prefix_dir = "/etc/hbase"
 stack_version_unformatted = status_params.stack_version_unformatted
 stack_version_formatted = status_params.stack_version_formatted
 stack_root = status_params.stack_root
+stack_supports_ranger_kerberos = stack_version_formatted and check_stack_feature(StackFeature.RANGER_KERBEROS_SUPPORT, stack_version_formatted)
 
 # hadoop default parameters
 hadoop_bin_dir = stack_select.get_hadoop_dir("bin")
@@ -336,6 +337,27 @@ if has_ranger_admin:
     'assetType': '2'
   }
 
+  if stack_supports_ranger_kerberos and security_enabled:
+    hbase_ranger_plugin_config['policydownload.auth.users'] = hbase_user
+    hbase_ranger_plugin_config['tag.download.auth.users'] = hbase_user
+    hbase_ranger_plugin_config['policy.grant.revoke.auth.users'] = hbase_user
+
+
+    hbase_ranger_plugin_repo = {
+      'isEnabled': 'true',
+      'configs': hbase_ranger_plugin_config,
+      'description': 'hbase repo',
+      'name': repo_name,
+      'type': 'hbase'
+    }
+
+  if 'hbase-master' in component_directory.lower():
+    ranger_hbase_principal = master_jaas_princ
+    ranger_hbase_keytab = master_keytab_path
+  else:
+    ranger_hbase_principal = regionserver_jaas_princ
+    ranger_hbase_keytab = regionserver_keytab_path
+
   xa_audit_db_is_enabled = False
   ranger_audit_solr_urls = config['configurations']['ranger-admin-site']['ranger.audit.solr.urls']
   if xml_configurations_supported and stack_supports_ranger_audit_db:

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/setup_ranger_hbase.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/setup_ranger_hbase.py b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/setup_ranger_hbase.py
index ffd0715..864d937 100644
--- a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/setup_ranger_hbase.py
+++ b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/setup_ranger_hbase.py
@@ -21,14 +21,10 @@ from resource_management.core.logger import Logger
 
 def setup_ranger_hbase(upgrade_type=None):
   import params
-  
+
   if params.has_ranger_admin:
 
-    if params.xml_configurations_supported:
-      from resource_management.libraries.functions.setup_ranger_plugin_xml import setup_ranger_plugin
-    else:
-      from resource_management.libraries.functions.setup_ranger_plugin import setup_ranger_plugin
-    
+
     stack_version = None
 
     if upgrade_type is not None:
@@ -66,8 +62,33 @@ def setup_ranger_hbase(upgrade_type=None):
       )
       params.HdfsResource(None, action="execute")
 
-    setup_ranger_plugin('hbase-client', 'hbase', 
-                        params.downloaded_custom_connector, params.driver_curl_source,
+    if params.xml_configurations_supported:
+      api_version=None
+      if params.stack_supports_ranger_kerberos and params.security_enabled:
+        api_version='v2'
+      from resource_management.libraries.functions.setup_ranger_plugin_xml import setup_ranger_plugin
+      setup_ranger_plugin('hbase-client', 'hbase', params.downloaded_custom_connector, params.driver_curl_source,
+                          params.driver_curl_target, params.java64_home,
+                          params.repo_name, params.hbase_ranger_plugin_repo,
+                          params.ranger_env, params.ranger_plugin_properties,
+                          params.policy_user, params.policymgr_mgr_url,
+                          params.enable_ranger_hbase, conf_dict=params.hbase_conf_dir,
+                          component_user=params.hbase_user, component_group=params.user_group, cache_service_list=['hbaseMaster', 'hbaseRegional'],
+                          plugin_audit_properties=params.config['configurations']['ranger-hbase-audit'], plugin_audit_attributes=params.config['configuration_attributes']['ranger-hbase-audit'],
+                          plugin_security_properties=params.config['configurations']['ranger-hbase-security'], plugin_security_attributes=params.config['configuration_attributes']['ranger-hbase-security'],
+                          plugin_policymgr_ssl_properties=params.config['configurations']['ranger-hbase-policymgr-ssl'], plugin_policymgr_ssl_attributes=params.config['configuration_attributes']['ranger-hbase-policymgr-ssl'],
+                          component_list=['hbase-client', 'hbase-master', 'hbase-regionserver'], audit_db_is_enabled=params.xa_audit_db_is_enabled,
+                          credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password,
+                          ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
+                          stack_version_override = stack_version, skip_if_rangeradmin_down= not params.retryAble, api_version=api_version,
+                          is_security_enabled = params.security_enabled,
+                          is_stack_supports_ranger_kerberos = params.stack_supports_ranger_kerberos if params.security_enabled else None,
+                          component_user_principal=params.ranger_hbase_principal if params.security_enabled else None,
+                          component_user_keytab=params.ranger_hbase_keytab if params.security_enabled else None)
+
+    else:
+      from resource_management.libraries.functions.setup_ranger_plugin import setup_ranger_plugin
+      setup_ranger_plugin('hbase-client', 'hbase', params.downloaded_custom_connector, params.driver_curl_source,
                         params.driver_curl_target, params.java64_home,
                         params.repo_name, params.hbase_ranger_plugin_repo,
                         params.ranger_env, params.ranger_plugin_properties,
@@ -78,7 +99,7 @@ def setup_ranger_hbase(upgrade_type=None):
                         plugin_security_properties=params.config['configurations']['ranger-hbase-security'], plugin_security_attributes=params.config['configuration_attributes']['ranger-hbase-security'],
                         plugin_policymgr_ssl_properties=params.config['configurations']['ranger-hbase-policymgr-ssl'], plugin_policymgr_ssl_attributes=params.config['configuration_attributes']['ranger-hbase-policymgr-ssl'],
                         component_list=['hbase-client', 'hbase-master', 'hbase-regionserver'], audit_db_is_enabled=params.xa_audit_db_is_enabled,
-                        credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password, 
+                        credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password,
                         ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
                         stack_version_override = stack_version, skip_if_rangeradmin_down= not params.retryAble)
   else:

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params_linux.py
index e6fd32c..a066dbd 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params_linux.py
@@ -55,6 +55,7 @@ stack_version_unformatted = config['hostLevelParams']['stack_version']
 stack_version_formatted = format_stack_version(stack_version_unformatted)
 agent_stack_retry_on_unavailability = config['hostLevelParams']['agent_stack_retry_on_unavailability']
 agent_stack_retry_count = expect("/hostLevelParams/agent_stack_retry_count", int)
+stack_supports_ranger_kerberos = stack_version_formatted and check_stack_feature(StackFeature.RANGER_KERBEROS_SUPPORT, stack_version_formatted)
 
 # there is a stack upgrade which has not yet been finalized; it's currently suspended
 upgrade_suspended = default("roleParams/upgrade_suspended", False)
@@ -484,6 +485,18 @@ if has_ranger_admin:
     'repositoryType': 'hdfs',
     'assetType': '1'
   }
+  if stack_supports_ranger_kerberos and security_enabled:
+    hdfs_ranger_plugin_config['policydownload.auth.users'] = hdfs_user
+    hdfs_ranger_plugin_config['tag.download.auth.users'] = hdfs_user
+
+    hdfs_ranger_plugin_repo = {
+      'isEnabled': 'true',
+      'configs': hdfs_ranger_plugin_config,
+      'description': 'hdfs repo',
+      'name': repo_name,
+      'type': 'hdfs'
+    }
+
   xa_audit_db_is_enabled = False
   ranger_audit_solr_urls = config['configurations']['ranger-admin-site']['ranger.audit.solr.urls']
   if xml_configurations_supported and stack_supports_ranger_audit_db:

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/setup_ranger_hdfs.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/setup_ranger_hdfs.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/setup_ranger_hdfs.py
index 858044c..040c69c 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/setup_ranger_hdfs.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/setup_ranger_hdfs.py
@@ -31,10 +31,6 @@ def setup_ranger_hdfs(upgrade_type=None):
 
   if params.has_ranger_admin:
 
-    if params.xml_configurations_supported:
-      from resource_management.libraries.functions.setup_ranger_plugin_xml import setup_ranger_plugin
-    else:
-      from resource_management.libraries.functions.setup_ranger_plugin import setup_ranger_plugin
 
     stack_version = None
 
@@ -46,21 +42,49 @@ def setup_ranger_hdfs(upgrade_type=None):
     else:
       Logger.info("HDFS: Setup ranger: command retry not enabled thus skipping if ranger admin is down !")
 
-    setup_ranger_plugin('hadoop-client', 'hdfs',
-                        params.downloaded_custom_connector, params.driver_curl_source,
-                        params.driver_curl_target, params.java_home,
-                        params.repo_name, params.hdfs_ranger_plugin_repo,
-                        params.ranger_env, params.ranger_plugin_properties,
-                        params.policy_user, params.policymgr_mgr_url,
-                        params.enable_ranger_hdfs, conf_dict=params.hadoop_conf_dir,
-                        component_user=params.hdfs_user, component_group=params.user_group, cache_service_list=['hdfs'],
-                        plugin_audit_properties=params.config['configurations']['ranger-hdfs-audit'], plugin_audit_attributes=params.config['configuration_attributes']['ranger-hdfs-audit'],
-                        plugin_security_properties=params.config['configurations']['ranger-hdfs-security'], plugin_security_attributes=params.config['configuration_attributes']['ranger-hdfs-security'],
-                        plugin_policymgr_ssl_properties=params.config['configurations']['ranger-hdfs-policymgr-ssl'], plugin_policymgr_ssl_attributes=params.config['configuration_attributes']['ranger-hdfs-policymgr-ssl'],
-                        component_list=['hadoop-client'], audit_db_is_enabled=params.xa_audit_db_is_enabled,
-                        credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password, 
-                        ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
-                        stack_version_override = stack_version, skip_if_rangeradmin_down= not params.retryAble)
+
+    if params.xml_configurations_supported:
+        from resource_management.libraries.functions.setup_ranger_plugin_xml import setup_ranger_plugin
+        api_version=None
+        if params.stack_supports_ranger_kerberos and params.security_enabled:
+          api_version='v2'
+        setup_ranger_plugin('hadoop-client', 'hdfs',
+                             params.downloaded_custom_connector, params.driver_curl_source,
+                             params.driver_curl_target, params.java_home,
+                             params.repo_name, params.hdfs_ranger_plugin_repo,
+                             params.ranger_env, params.ranger_plugin_properties,
+                             params.policy_user, params.policymgr_mgr_url,
+                             params.enable_ranger_hdfs, conf_dict=params.hadoop_conf_dir,
+                             component_user=params.hdfs_user, component_group=params.user_group, cache_service_list=['hdfs'],
+                             plugin_audit_properties=params.config['configurations']['ranger-hdfs-audit'], plugin_audit_attributes=params.config['configuration_attributes']['ranger-hdfs-audit'],
+                             plugin_security_properties=params.config['configurations']['ranger-hdfs-security'], plugin_security_attributes=params.config['configuration_attributes']['ranger-hdfs-security'],
+                             plugin_policymgr_ssl_properties=params.config['configurations']['ranger-hdfs-policymgr-ssl'], plugin_policymgr_ssl_attributes=params.config['configuration_attributes']['ranger-hdfs-policymgr-ssl'],
+                             component_list=['hadoop-client'], audit_db_is_enabled=params.xa_audit_db_is_enabled,
+                             credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password,
+                             ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
+                             api_version=api_version ,stack_version_override = stack_version, skip_if_rangeradmin_down= not params.retryAble,
+                             is_security_enabled = params.security_enabled,
+                             is_stack_supports_ranger_kerberos = params.stack_supports_ranger_kerberos,
+                             component_user_principal=params.nn_principal_name if params.security_enabled else None,
+                             component_user_keytab=params.nn_keytab if params.security_enabled else None)
+    else:
+        from resource_management.libraries.functions.setup_ranger_plugin import setup_ranger_plugin
+
+        setup_ranger_plugin('hadoop-client', 'hdfs',
+                            params.downloaded_custom_connector, params.driver_curl_source,
+                            params.driver_curl_target, params.java_home,
+                            params.repo_name, params.hdfs_ranger_plugin_repo,
+                            params.ranger_env, params.ranger_plugin_properties,
+                            params.policy_user, params.policymgr_mgr_url,
+                            params.enable_ranger_hdfs, conf_dict=params.hadoop_conf_dir,
+                            component_user=params.hdfs_user, component_group=params.user_group, cache_service_list=['hdfs'],
+                            plugin_audit_properties=params.config['configurations']['ranger-hdfs-audit'], plugin_audit_attributes=params.config['configuration_attributes']['ranger-hdfs-audit'],
+                            plugin_security_properties=params.config['configurations']['ranger-hdfs-security'], plugin_security_attributes=params.config['configuration_attributes']['ranger-hdfs-security'],
+                            plugin_policymgr_ssl_properties=params.config['configurations']['ranger-hdfs-policymgr-ssl'], plugin_policymgr_ssl_attributes=params.config['configuration_attributes']['ranger-hdfs-policymgr-ssl'],
+                            component_list=['hadoop-client'], audit_db_is_enabled=params.xa_audit_db_is_enabled,
+                            credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password,
+                            ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
+                            stack_version_override = stack_version, skip_if_rangeradmin_down= not params.retryAble)
 
     if stack_version and params.upgrade_direction == Direction.UPGRADE:
       # when upgrading to stack remove_ranger_hdfs_plugin_env, this env file must be removed

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/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 d0c3b3a..22e1b55 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
@@ -77,6 +77,7 @@ downgrade_from_version = default("/commandParams/downgrade_from_version", None)
 
 # Upgrade direction
 upgrade_direction = default("/commandParams/upgrade_direction", None)
+stack_supports_ranger_kerberos = stack_version_formatted_major and check_stack_feature(StackFeature.RANGER_KERBEROS_SUPPORT, stack_version_formatted_major)
 
 hadoop_bin_dir = "/usr/bin"
 hadoop_home = '/usr'
@@ -641,6 +642,20 @@ if has_ranger_admin:
     'assetType': '3'
   }
 
+  if stack_supports_ranger_kerberos and security_enabled:
+    hive_ranger_plugin_config['policydownload.auth.users'] = hive_user
+    hive_ranger_plugin_config['tag.download.auth.users'] = hive_user
+    hive_ranger_plugin_config['policy.grant.revoke.auth.users'] = hive_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 = unicode(config['configurations']['admin-properties']['audit_db_password']) if stack_supports_ranger_audit_db else None
   ranger_audit_solr_urls = config['configurations']['ranger-admin-site']['ranger.audit.solr.urls']

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/setup_ranger_hive.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/setup_ranger_hive.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/setup_ranger_hive.py
index f51dbab..92eaaab 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/setup_ranger_hive.py
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/setup_ranger_hive.py
@@ -24,11 +24,6 @@ def setup_ranger_hive(upgrade_type = None):
 
   if params.has_ranger_admin:
 
-    if params.xml_configurations_supported: 
-      from resource_management.libraries.functions.setup_ranger_plugin_xml import setup_ranger_plugin
-    else:
-      from resource_management.libraries.functions.setup_ranger_plugin import setup_ranger_plugin
-    
     stack_version = None
 
     if upgrade_type is not None:
@@ -58,7 +53,33 @@ def setup_ranger_hive(upgrade_type = None):
       )
       params.HdfsResource(None, action="execute")
 
-    setup_ranger_plugin('hive-server2', 'hive', 
+    if params.xml_configurations_supported:
+      api_version=None
+      if params.stack_supports_ranger_kerberos and params.security_enabled:
+        api_version='v2'
+      from resource_management.libraries.functions.setup_ranger_plugin_xml import setup_ranger_plugin
+      setup_ranger_plugin('hive-server2', 'hive',
+                          params.ranger_downloaded_custom_connector, params.ranger_driver_curl_source,
+                          params.ranger_driver_curl_target, params.java64_home,
+                          params.repo_name, params.hive_ranger_plugin_repo,
+                          params.ranger_env, params.ranger_plugin_properties,
+                          params.policy_user, params.policymgr_mgr_url,
+                          params.enable_ranger_hive, conf_dict=params.hive_server_conf_dir,
+                          component_user=params.hive_user, component_group=params.user_group, cache_service_list=['hiveServer2'],
+                          plugin_audit_properties=params.config['configurations']['ranger-hive-audit'], plugin_audit_attributes=params.config['configuration_attributes']['ranger-hive-audit'],
+                          plugin_security_properties=params.config['configurations']['ranger-hive-security'], plugin_security_attributes=params.config['configuration_attributes']['ranger-hive-security'],
+                          plugin_policymgr_ssl_properties=params.config['configurations']['ranger-hive-policymgr-ssl'], plugin_policymgr_ssl_attributes=params.config['configuration_attributes']['ranger-hive-policymgr-ssl'],
+                          component_list=['hive-client', 'hive-metastore', 'hive-server2'], audit_db_is_enabled=params.xa_audit_db_is_enabled,
+                          credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password,
+                          ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
+                          stack_version_override = stack_version, skip_if_rangeradmin_down= not params.retryAble, api_version=api_version,
+                          is_security_enabled = params.security_enabled,
+                          is_stack_supports_ranger_kerberos = params.stack_supports_ranger_kerberos,
+                          component_user_principal=params.hive_server_principal if params.security_enabled else None,
+                          component_user_keytab=params.hive_server2_keytab if params.security_enabled else None)
+    else:
+      from resource_management.libraries.functions.setup_ranger_plugin import setup_ranger_plugin
+      setup_ranger_plugin('hive-server2', 'hive',
                         params.ranger_downloaded_custom_connector, params.ranger_driver_curl_source,
                         params.ranger_driver_curl_target, params.java64_home,
                         params.repo_name, params.hive_ranger_plugin_repo,
@@ -70,7 +91,7 @@ def setup_ranger_hive(upgrade_type = None):
                         plugin_security_properties=params.config['configurations']['ranger-hive-security'], plugin_security_attributes=params.config['configuration_attributes']['ranger-hive-security'],
                         plugin_policymgr_ssl_properties=params.config['configurations']['ranger-hive-policymgr-ssl'], plugin_policymgr_ssl_attributes=params.config['configuration_attributes']['ranger-hive-policymgr-ssl'],
                         component_list=['hive-client', 'hive-metastore', 'hive-server2'], audit_db_is_enabled=params.xa_audit_db_is_enabled,
-                        credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password, 
+                        credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password,
                         ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
                         stack_version_override = stack_version, skip_if_rangeradmin_down= not params.retryAble)
   else:

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/params.py b/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/params.py
index d4ee6f9..0a2796b 100644
--- a/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/params.py
@@ -53,6 +53,7 @@ host_sys_prepped = default("/hostLevelParams/host_sys_prepped", False)
 stack_version_unformatted = config['hostLevelParams']['stack_version']
 stack_version_formatted = format_stack_version(stack_version_unformatted)
 upgrade_direction = default("/commandParams/upgrade_direction", None)
+stack_supports_ranger_kerberos = stack_version_formatted and check_stack_feature(StackFeature.RANGER_KERBEROS_SUPPORT, stack_version_formatted)
 
 # 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
@@ -209,6 +210,12 @@ if has_ranger_admin and is_supported_kafka_ranger:
     'type': 'kafka',
     'assetType': '1'
   }
+
+  if stack_supports_ranger_kerberos and security_enabled:
+    ranger_plugin_config['policydownload.auth.users'] = kafka_user
+    ranger_plugin_config['tag.download.auth.users'] = kafka_user
+
+
   #For curl command in ranger plugin to get db connector
   jdk_location = config['hostLevelParams']['jdk_location']
   java_share_dir = '/usr/share/java'

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/setup_ranger_kafka.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/setup_ranger_kafka.py b/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/setup_ranger_kafka.py
index a99dc76..3a3ecfe 100644
--- a/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/setup_ranger_kafka.py
+++ b/ambari-server/src/main/resources/common-services/KAFKA/0.8.1.2.2/package/scripts/setup_ranger_kafka.py
@@ -50,7 +50,7 @@ def setup_ranger_kafka():
         )
         params.HdfsResource(None, action="execute")
 
-    setup_ranger_plugin('kafka-broker', 'kafka', 
+    setup_ranger_plugin('kafka-broker', 'kafka',
                         params.downloaded_custom_connector, params.driver_curl_source,
                         params.driver_curl_target, params.java64_home,
                         params.repo_name, params.kafka_ranger_plugin_repo,
@@ -64,7 +64,11 @@ def setup_ranger_kafka():
                         component_list=['kafka-broker'], audit_db_is_enabled=params.xa_audit_db_is_enabled,
                         credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password, 
                         ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
-                        api_version = 'v2', skip_if_rangeradmin_down= not params.retryAble)
+                        api_version = 'v2', skip_if_rangeradmin_down= not params.retryAble,
+                        is_security_enabled = params.security_enabled,
+                        is_stack_supports_ranger_kerberos = params.stack_supports_ranger_kerberos,
+                        component_user_principal=params.kafka_jaas_principal if params.security_enabled else None,
+                        component_user_keytab=params.kafka_keytab_path if params.security_enabled else None)
     
     if params.enable_ranger_kafka: 
       Execute(('cp', '--remove-destination', params.setup_ranger_env_sh_source, params.setup_ranger_env_sh_target),

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params_linux.py
index d572aff..1aea9bf 100644
--- a/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params_linux.py
@@ -52,6 +52,7 @@ version_formatted = format_stack_version(version)
 # E.g., 2.3
 stack_version_unformatted = config['hostLevelParams']['stack_version']
 stack_version_formatted = format_stack_version(stack_version_unformatted)
+stack_supports_ranger_kerberos = stack_version_formatted and check_stack_feature(StackFeature.RANGER_KERBEROS_SUPPORT, stack_version_formatted)
 
 # This is the version whose state is CURRENT. During an RU, this is the source version.
 # DO NOT format it since we need the build number too.
@@ -318,7 +319,21 @@ if has_ranger_admin:
     'repositoryType': 'knox',
     'assetType': '5',
     }
-  
+
+  if stack_supports_ranger_kerberos and security_enabled:
+    knox_ranger_plugin_config['policydownload.auth.users'] = knox_user
+    knox_ranger_plugin_config['tag.download.auth.users'] = knox_user
+
+    knox_ranger_plugin_repo = {
+      'isEnabled': 'true',
+      'configs': knox_ranger_plugin_config,
+      'description': 'knox repo',
+      'name': repo_name,
+      'type': 'knox'
+    }
+
+
+
   xa_audit_db_is_enabled = False
   ranger_audit_solr_urls = config['configurations']['ranger-admin-site']['ranger.audit.solr.urls']
   if xml_configurations_supported and stack_supports_ranger_audit_db:

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/setup_ranger_knox.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/setup_ranger_knox.py b/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/setup_ranger_knox.py
index 13987c8..c5f8940 100644
--- a/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/setup_ranger_knox.py
+++ b/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/setup_ranger_knox.py
@@ -21,14 +21,10 @@ from resource_management.core.logger import Logger
 
 def setup_ranger_knox(upgrade_type=None):
   import params
-  
+
   if params.has_ranger_admin:
 
-    if params.xml_configurations_supported:
-      from resource_management.libraries.functions.setup_ranger_plugin_xml import setup_ranger_plugin
-    else:
-      from resource_management.libraries.functions.setup_ranger_plugin import setup_ranger_plugin
-    
+
     stack_version = None
     if upgrade_type is not None:
       stack_version = params.version
@@ -58,7 +54,33 @@ def setup_ranger_knox(upgrade_type=None):
         )
         params.HdfsResource(None, action="execute")
 
-    setup_ranger_plugin('knox-server', 'knox',
+    if params.xml_configurations_supported:
+      api_version=None
+      if params.stack_supports_ranger_kerberos and params.security_enabled:
+        api_version='v2'
+      from resource_management.libraries.functions.setup_ranger_plugin_xml import setup_ranger_plugin
+      setup_ranger_plugin('knox-server', 'knox',
+                          params.downloaded_custom_connector, params.driver_curl_source,
+                          params.driver_curl_target, params.java_home,
+                          params.repo_name, params.knox_ranger_plugin_repo,
+                          params.ranger_env, params.ranger_plugin_properties,
+                          params.policy_user, params.policymgr_mgr_url,
+                          params.enable_ranger_knox, conf_dict=params.knox_conf_dir,
+                          component_user=params.knox_user, component_group=params.knox_group, cache_service_list=['knox'],
+                          plugin_audit_properties=params.config['configurations']['ranger-knox-audit'], plugin_audit_attributes=params.config['configuration_attributes']['ranger-knox-audit'],
+                          plugin_security_properties=params.config['configurations']['ranger-knox-security'], plugin_security_attributes=params.config['configuration_attributes']['ranger-knox-security'],
+                          plugin_policymgr_ssl_properties=params.config['configurations']['ranger-knox-policymgr-ssl'], plugin_policymgr_ssl_attributes=params.config['configuration_attributes']['ranger-knox-policymgr-ssl'],
+                          component_list=['knox-server'], audit_db_is_enabled=params.xa_audit_db_is_enabled,
+                          credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password,
+                          ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
+                          stack_version_override = stack_version, skip_if_rangeradmin_down= not params.retryAble,api_version=api_version,
+                          is_security_enabled = params.security_enabled,
+                          is_stack_supports_ranger_kerberos = params.stack_supports_ranger_kerberos,
+                          component_user_principal=params.knox_principal_name if params.security_enabled else None,
+                          component_user_keytab=params.knox_keytab_path if params.security_enabled else None)
+    else:
+      from resource_management.libraries.functions.setup_ranger_plugin import setup_ranger_plugin
+      setup_ranger_plugin('knox-server', 'knox',
                         params.downloaded_custom_connector, params.driver_curl_source,
                         params.driver_curl_target, params.java_home,
                         params.repo_name, params.knox_ranger_plugin_repo,
@@ -74,4 +96,4 @@ def setup_ranger_knox(upgrade_type=None):
                         ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
                         stack_version_override = stack_version, skip_if_rangeradmin_down= not params.retryAble)
   else:
-    Logger.info('Ranger admin not installed')
\ No newline at end of file
+    Logger.info('Ranger admin not installed')

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-server/src/main/resources/common-services/RANGER/0.4.0/package/scripts/setup_ranger_xml.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/RANGER/0.4.0/package/scripts/setup_ranger_xml.py b/ambari-server/src/main/resources/common-services/RANGER/0.4.0/package/scripts/setup_ranger_xml.py
index 914d63d..ec0eea1 100644
--- a/ambari-server/src/main/resources/common-services/RANGER/0.4.0/package/scripts/setup_ranger_xml.py
+++ b/ambari-server/src/main/resources/common-services/RANGER/0.4.0/package/scripts/setup_ranger_xml.py
@@ -533,12 +533,21 @@ def ranger_credential_helper(lib_path, alias_key, alias_value, file_path):
 def create_core_site_xml(conf_dir):
   import params
 
-  if params.stack_supports_ranger_kerberos and params.security_enabled and params.has_namenode:
-    XmlConfig("core-site.xml",
-      conf_dir=conf_dir,
-      configurations=params.config['configurations']['core-site'],
-      configuration_attributes=params.config['configuration_attributes']['core-site'],
-      owner=params.unix_user,
-      group=params.unix_group,
-      mode=0644
-    )
+  if params.stack_supports_ranger_kerberos:
+    if params.has_namenode:
+      XmlConfig("core-site.xml",
+                conf_dir=conf_dir,
+                configurations=params.config['configurations']['core-site'],
+                configuration_attributes=params.config['configuration_attributes']['core-site'],
+                owner=params.unix_user,
+                group=params.unix_group,
+                mode=0644
+      )
+    else:
+      Logger.warning('HDFS service not installed. Creating blank core-site.xml file.')
+      File(format('{conf_dir}/core-site.xml'),
+           content = '<configuration></configuration>',
+           owner = params.unix_user,
+           group = params.unix_group,
+           mode=0644
+      )
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_linux.py
index 3f9b936..22da38f 100644
--- a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/params_linux.py
@@ -62,6 +62,7 @@ stack_version_formatted = status_params.stack_version_formatted
 stack_supports_ru = stack_version_formatted and check_stack_feature(StackFeature.ROLLING_UPGRADE, stack_version_formatted)
 stack_supports_storm_kerberos = stack_version_formatted and check_stack_feature(StackFeature.STORM_KERBEROS, stack_version_formatted)
 stack_supports_storm_ams = stack_version_formatted and check_stack_feature(StackFeature.STORM_AMS, stack_version_formatted)
+stack_supports_ranger_kerberos = stack_version_formatted and check_stack_feature(StackFeature.RANGER_KERBEROS_SUPPORT, stack_version_formatted)
 
 # default hadoop params
 rest_lib_dir = "/usr/lib/storm/contrib/storm-rest"
@@ -287,6 +288,25 @@ if has_ranger_admin:
     'assetType': '6'
   }
 
+  if stack_supports_ranger_kerberos and security_enabled:
+    storm_ranger_plugin_config['policydownload.auth.users'] = storm_user
+    storm_ranger_plugin_config['tag.download.auth.users'] = storm_user
+
+    storm_ranger_plugin_repo = {
+      'isEnabled': 'true',
+      'configs': storm_ranger_plugin_config,
+      'description': 'storm repo',
+      'name': repo_name,
+      'type': 'storm'
+    }
+
+  if 'storm-nimbus' in status_params.component_directory.lower():
+    ranger_storm_principal = nimbus_jaas_principal
+    ranger_storm_keytab = nimbus_keytab_path
+  else:
+    ranger_storm_principal = storm_ui_jaas_principal
+    ranger_storm_keytab = storm_ui_keytab_path
+
   xa_audit_db_is_enabled = False
   ranger_audit_solr_urls = config['configurations']['ranger-admin-site']['ranger.audit.solr.urls']
   if xml_configurations_supported and stack_supports_ranger_audit_db:

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/setup_ranger_storm.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/setup_ranger_storm.py b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/setup_ranger_storm.py
index bef1f02..ba4c777 100644
--- a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/setup_ranger_storm.py
+++ b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/package/scripts/setup_ranger_storm.py
@@ -24,14 +24,8 @@ def setup_ranger_storm(upgrade_type=None):
   :param upgrade_type: Upgrade Type such as "rolling" or "nonrolling"
   """
   import params
-
   if params.has_ranger_admin and params.security_enabled:
 
-    if params.xml_configurations_supported:
-      from resource_management.libraries.functions.setup_ranger_plugin_xml import setup_ranger_plugin
-    else:
-      from resource_management.libraries.functions.setup_ranger_plugin import setup_ranger_plugin
-    
     stack_version = None
     if upgrade_type is not None:
       stack_version = params.version
@@ -61,7 +55,34 @@ def setup_ranger_storm(upgrade_type=None):
         )
         params.HdfsResource(None, action="execute")
 
-    setup_ranger_plugin('storm-nimbus', 'storm',
+    if params.xml_configurations_supported:
+      api_version=None
+      if params.stack_supports_ranger_kerberos and params.security_enabled:
+        Logger.info('setting stack_version as v2')
+        api_version='v2'
+      from resource_management.libraries.functions.setup_ranger_plugin_xml import setup_ranger_plugin
+      setup_ranger_plugin('storm-nimbus', 'storm',
+                          params.downloaded_custom_connector, params.driver_curl_source,
+                          params.driver_curl_target, params.java64_home,
+                          params.repo_name, params.storm_ranger_plugin_repo,
+                          params.ranger_env, params.ranger_plugin_properties,
+                          params.policy_user, params.policymgr_mgr_url,
+                          params.enable_ranger_storm, conf_dict=params.conf_dir,
+                          component_user=params.storm_user, component_group=params.user_group, cache_service_list=['storm'],
+                          plugin_audit_properties=params.config['configurations']['ranger-storm-audit'], plugin_audit_attributes=params.config['configuration_attributes']['ranger-storm-audit'],
+                          plugin_security_properties=params.config['configurations']['ranger-storm-security'], plugin_security_attributes=params.config['configuration_attributes']['ranger-storm-security'],
+                          plugin_policymgr_ssl_properties=params.config['configurations']['ranger-storm-policymgr-ssl'], plugin_policymgr_ssl_attributes=params.config['configuration_attributes']['ranger-storm-policymgr-ssl'],
+                          component_list=['storm-client', 'storm-nimbus'], audit_db_is_enabled=params.xa_audit_db_is_enabled,
+                          credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password,
+                          ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
+                          stack_version_override = stack_version, skip_if_rangeradmin_down= not params.retryAble,api_version=api_version,
+                          is_security_enabled = params.security_enabled,
+                          is_stack_supports_ranger_kerberos = params.stack_supports_ranger_kerberos,
+                          component_user_principal=params.ranger_storm_principal if params.security_enabled else None,
+                          component_user_keytab=params.ranger_storm_keytab if params.security_enabled else None)
+    else:
+      from resource_management.libraries.functions.setup_ranger_plugin import setup_ranger_plugin
+      setup_ranger_plugin('storm-nimbus', 'storm',
                         params.downloaded_custom_connector, params.driver_curl_source,
                         params.driver_curl_target, params.java64_home,
                         params.repo_name, params.storm_ranger_plugin_repo,

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/params_linux.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/params_linux.py b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/params_linux.py
index 8fbefb2..efa303c 100644
--- a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/params_linux.py
+++ b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/params_linux.py
@@ -65,6 +65,7 @@ stack_version_formatted = functions.get_stack_version('hadoop-yarn-resourcemanag
 
 stack_supports_ru = stack_version_formatted_major and check_stack_feature(StackFeature.ROLLING_UPGRADE, stack_version_formatted_major)
 stack_supports_timeline_state_store = stack_version_formatted_major and check_stack_feature(StackFeature.TIMELINE_STATE_STORE, stack_version_formatted_major)
+stack_supports_ranger_kerberos = stack_version_formatted_major and check_stack_feature(StackFeature.RANGER_KERBEROS_SUPPORT, stack_version_formatted_major)
 
 # New Cluster Stack Version that is defined during the RESTART of a Stack Upgrade.
 # It cannot be used during the initial Cluser Install because the version is not yet known.
@@ -252,10 +253,10 @@ yarn_timelineservice_kinit_cmd = ""
 nodemanager_kinit_cmd = ""
 
 if security_enabled:
-  _rm_principal_name = config['configurations']['yarn-site']['yarn.resourcemanager.principal']
-  _rm_principal_name = _rm_principal_name.replace('_HOST',hostname.lower())
-  _rm_keytab = config['configurations']['yarn-site']['yarn.resourcemanager.keytab']
-  rm_kinit_cmd = format("{kinit_path_local} -kt {_rm_keytab} {_rm_principal_name};")
+  rm_principal_name = config['configurations']['yarn-site']['yarn.resourcemanager.principal']
+  rm_principal_name = rm_principal_name.replace('_HOST',hostname.lower())
+  rm_keytab = config['configurations']['yarn-site']['yarn.resourcemanager.keytab']
+  rm_kinit_cmd = format("{kinit_path_local} -kt {rm_keytab} {rm_principal_name};")
 
   # YARN timeline security options
   if has_ats:
@@ -413,6 +414,11 @@ if has_ranger_admin:
       'type': 'yarn',
       'assetType': '1'
     }
+
+    if stack_supports_ranger_kerberos and security_enabled:
+      ranger_plugin_config['policydownload.auth.users'] = yarn_user
+      ranger_plugin_config['tag.download.auth.users'] = yarn_user
+
     #For curl command in ranger plugin to get db connector
     jdk_location = config['hostLevelParams']['jdk_location']
     java_share_dir = '/usr/share/java'

http://git-wip-us.apache.org/repos/asf/ambari/blob/e74bfa4a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/setup_ranger_yarn.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/setup_ranger_yarn.py b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/setup_ranger_yarn.py
index 21fe8e1..1f61d41 100644
--- a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/setup_ranger_yarn.py
+++ b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/setup_ranger_yarn.py
@@ -47,7 +47,7 @@ def setup_ranger_yarn():
       )
       params.HdfsResource(None, action="execute")
 
-    setup_ranger_plugin('hadoop-yarn-resourcemanager', 'yarn', 
+    setup_ranger_plugin('hadoop-yarn-resourcemanager', 'yarn',
                         params.downloaded_custom_connector, params.driver_curl_source,
                         params.driver_curl_target, params.java64_home,
                         params.repo_name, params.yarn_ranger_plugin_repo,
@@ -61,7 +61,11 @@ def setup_ranger_yarn():
                         component_list=['hadoop-yarn-resourcemanager'], audit_db_is_enabled=params.xa_audit_db_is_enabled,
                         credential_file=params.credential_file, xa_audit_db_password=params.xa_audit_db_password, 
                         ssl_truststore_password=params.ssl_truststore_password, ssl_keystore_password=params.ssl_keystore_password,
-                        api_version = 'v2', skip_if_rangeradmin_down= not params.retryAble
-      )                 
+                        api_version = 'v2', skip_if_rangeradmin_down= not params.retryAble,
+                        is_security_enabled = params.security_enabled,
+                        is_stack_supports_ranger_kerberos = params.stack_supports_ranger_kerberos,
+                        component_user_principal=params.rm_principal_name if params.security_enabled else None,
+                        component_user_keytab=params.rm_keytab if params.security_enabled else None
+      )
   else:
     Logger.info('Ranger admin not installed')
\ No newline at end of file