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 2015/05/05 23:49:43 UTC

ambari git commit: AMBARI-10866. Implement Stack changes for Ranger Yarn Plugin integration (Gautam Borad via alejandro)

Repository: ambari
Updated Branches:
  refs/heads/trunk 3f6ef73b6 -> 260edad61


AMBARI-10866. Implement Stack changes for Ranger Yarn Plugin integration (Gautam Borad via alejandro)


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

Branch: refs/heads/trunk
Commit: 260edad611620998e1f4b5e494763a7531895272
Parents: 3f6ef73
Author: Alejandro Fernandez <af...@hortonworks.com>
Authored: Tue May 5 14:45:07 2015 -0700
Committer: Alejandro Fernandez <af...@hortonworks.com>
Committed: Tue May 5 14:45:07 2015 -0700

----------------------------------------------------------------------
 .../libraries/functions/ranger_functions_v2.py  | 243 +++++++++++++++++++
 .../libraries/functions/setup_ranger_plugin.py  |  12 +-
 .../YARN/2.1.0.2.0/configuration/yarn-env.xml   |   6 +-
 .../2.1.0.2.0/package/scripts/params_linux.py   |  71 +++++-
 .../package/scripts/resourcemanager.py          |   4 +-
 .../package/scripts/setup_ranger_yarn.py        |  36 +++
 .../ranger-yarn-plugin-properties.xml           | 235 ++++++++++++++++++
 .../services/YARN/configuration/yarn-env.xml    |  29 +++
 .../python/stacks/2.0.6/configs/default.json    |   5 +-
 .../python/stacks/2.0.6/configs/secured.json    |   3 +-
 .../test/python/stacks/2.1/configs/default.json |   5 +-
 .../test/python/stacks/2.1/configs/secured.json |   3 +-
 .../test/python/stacks/2.3/configs/default.json |   5 +
 13 files changed, 644 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/260edad6/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
new file mode 100644
index 0000000..da05773
--- /dev/null
+++ b/ambari-common/src/main/python/resource_management/libraries/functions/ranger_functions_v2.py
@@ -0,0 +1,243 @@
+#!/usr/bin/env python
+"""
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+"""
+import time
+import sys
+from StringIO import StringIO as BytesIO
+import json
+from resource_management.core.logger import Logger
+import urllib2, base64, httplib
+from resource_management.core.exceptions import Fail
+from resource_management.libraries.functions.format import format
+
+
+
+class RangeradminV2:
+  sInstance = None
+
+  def __init__(self, url='http://localhost:6080'):
+    
+    self.base_url = url
+    self.url_login = self.base_url + '/login.jsp'
+    self.url_login_post = self.base_url + '/j_spring_security_check'
+    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_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'
+
+    self.session = None
+    self.isLoggedIn = False
+
+  def get_repository_by_name_urllib2(self, name, component, status, usernamepassword):
+    """
+    :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. 
+    :return: Returns Ranger repository object if found otherwise None
+    """
+    try:
+      search_repo_url = self.url_repos_pub + "?name=" + name + "&type=" + component + "&status=" + status
+      request = urllib2.Request(search_repo_url)
+      base_64_string = base64.encodestring(usernamepassword).replace('\n', '')
+      request.add_header("Content-Type", "application/json")
+      request.add_header("Accept", "application/json")
+      request.add_header("Authorization", "Basic {0}".format(base_64_string))
+      result = urllib2.urlopen(request)
+      response_code = result.getcode()
+      response = json.loads(result.read())
+
+      if response_code == 200 and len(response) > 0:
+        for repo in response:
+          repo_dump = json.loads(json.JSONEncoder().encode(repo))
+          if repo_dump['name'] == name:
+            return repo_dump
+        return None
+      else:
+        return None
+    except urllib2.URLError, e:
+      if isinstance(e, urllib2.HTTPError):
+        Logger.error("HTTP Code: {0}".format(e.code))
+        Logger.error("HTTP Data: {0}".format(e.read()))
+      else:
+        Logger.error("Error : {0}".format(e.reason))
+      return None
+    except httplib.BadStatusLine:
+      Logger.error("Ranger Admin service is not reachable, please restart the service and then try again")
+      return None
+      
+    
+  def create_ranger_repository(self, component, repo_name, repo_properties, 
+                               ambari_ranger_admin, ambari_ranger_password,
+                               admin_uname, admin_password, policy_user):
+    response_code, response_recieved = self.check_ranger_login_urllib2(self.url_login, 'test:test')
+    repo_data = json.dumps(repo_properties)
+    
+    if response_code is not None and response_code == 200:
+      ambari_ranger_admin, ambari_ranger_password = self.create_ambari_admin_user(ambari_ranger_admin, ambari_ranger_password, format("{admin_uname}:{admin_password}"))
+      ambari_username_password_for_ranger = ambari_ranger_admin + ':' + ambari_ranger_password
+      if ambari_ranger_admin != '' and ambari_ranger_password != '':
+        repo = self.get_repository_by_name_urllib2(repo_name, component, 'true', ambari_username_password_for_ranger)
+        if repo and repo['name'] == repo_name:
+          Logger.info('{0} Repository exist'.format(component.title()))
+        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()))
+          else:
+            Logger.error('{0} Repository creation failed in Ranger admin'.format(component.title()))
+      else:
+        Logger.error('Ambari admin username and password are blank ')
+          
+  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. 
+    :return: Returns created Ranger repository object
+    """
+    try:
+      search_repo_url = self.url_repos_pub
+      base_64_string = base64.encodestring('{0}'.format(usernamepassword)).replace('\n', '')
+      headers = {
+        'Accept': 'application/json',
+        "Content-Type": "application/json"
+      }
+      request = urllib2.Request(search_repo_url, data, headers)
+      request.add_header("Authorization", "Basic {0}".format(base_64_string))
+      result = urllib2.urlopen(request)
+      response_code = result.getcode()
+      response = json.loads(json.JSONEncoder().encode(result.read()))
+
+      if response_code == 200:
+        Logger.info('Repository created Successfully')
+        return response
+      else:
+        Logger.error(response_code)
+        Logger.error(response)
+        return None
+    except urllib2.URLError, e:
+      if isinstance(e, urllib2.HTTPError):
+        Logger.error("HTTP Code: {0}".format(e.code))
+        Logger.error("HTTP Data: {0}".format(e.read()))
+      else:
+        Logger.error("Error: {0}".format(e.reason))
+      return None
+    except httplib.BadStatusLine:
+      Logger.error("Ranger Admin service is not reachable, please restart the service and then try again")
+      return None
+
+  def check_ranger_login_urllib2(self, url, usernamepassword):
+    """
+    :param url: ranger admin host url
+    :param usernamepassword: user credentials using which repository needs to be searched. 
+    :return: Returns login check response 
+    """
+    try:
+      request = urllib2.Request(url)
+      base_64_string = base64.encodestring(usernamepassword).replace('\n', '')
+      request.add_header("Content-Type", "application/json")
+      request.add_header("Accept", "application/json")
+      request.add_header("Authorization", "Basic {0}".format(base_64_string))
+      result = urllib2.urlopen(request)
+      response = result.read()
+      response_code = result.getcode()
+      return response_code, response
+    except urllib2.URLError, e:
+      if isinstance(e, urllib2.HTTPError):
+        Logger.error("HTTP Code: {0}".format(e.code))
+        Logger.error("HTTP Data: {0}".format(e.read()))
+      else:
+        Logger.error("Error : {0}".format(e.reason))
+      return None, None
+    except httplib.BadStatusLine, e:
+      Logger.error("Ranger Admin service is not reachable, please restart the service and then try again")
+      return None, 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: user password of user to be created 
+    :return: Returns user credentials if user exist otherwise rerutns credentials of  created user.
+    """
+    try:
+      url =  self.url_users + '?startIndex=0'
+      request = urllib2.Request(url)
+      base_64_string = base64.encodestring(usernamepassword).replace('\n', '')
+      request.add_header("Content-Type", "application/json")
+      request.add_header("Accept", "application/json")
+      request.add_header("Authorization", "Basic {0}".format(base_64_string))
+      result = urllib2.urlopen(request)
+      response_code =  result.getcode()
+      response = json.loads(result.read())
+      if response_code == 200 and len(response['vXUsers']) >= 0:
+        ambari_admin_username = ambari_admin_username
+        flag_ambari_admin_present = False
+        for vxuser in response['vXUsers']:
+          rangerlist_username = vxuser['name']
+          if rangerlist_username == ambari_admin_username:
+            flag_ambari_admin_present = True
+            break
+          else:
+            flag_ambari_admin_present = False
+
+        if flag_ambari_admin_present:
+          Logger.info(ambari_admin_username + ' user already exists, using existing user from configurations.')
+          return ambari_admin_username,ambari_admin_password
+        else:
+          Logger.info(ambari_admin_username + ' user is not present, creating user using given configurations')
+          url = self.url_sec_users
+          admin_user = dict()
+          admin_user['status'] = 1
+          admin_user['userRoleList'] = ['ROLE_SYS_ADMIN']
+          admin_user['name'] = ambari_admin_username
+          admin_user['password'] = ambari_admin_password
+          admin_user['description'] = ambari_admin_username
+          admin_user['firstName'] = ambari_admin_username
+          data =  json.dumps(admin_user)
+          base_64_string = base64.encodestring('{0}'.format(usernamepassword)).replace('\n', '')
+          headers = {
+            'Accept': 'application/json',
+            "Content-Type": "application/json"
+          }
+          request = urllib2.Request(url, data, headers)
+          request.add_header("Authorization", "Basic {0}".format(base_64_string))
+          result = urllib2.urlopen(request)
+          response_code =  result.getcode()
+          response = json.loads(json.JSONEncoder().encode(result.read()))
+          if response_code == 200 and response is not None:
+            Logger.info('Ambari admin user creation successful.')
+          else:
+            Logger.error('Ambari admin user creation failed,setting username and password as blank')
+            ambari_admin_username = ''
+            ambari_admin_password = ''
+          return ambari_admin_username,ambari_admin_password
+      else:
+        return '',''
+    except urllib2.URLError, e:
+      if isinstance(e, urllib2.HTTPError):
+        Logger.error("HTTP Code: {0}".format(e.code))
+        Logger.error("HTTP Data: {0}".format(e.read()))
+      else:
+        Logger.error("Error: {0}".format(e.reason))
+      return '',''
+    except httplib.BadStatusLine:
+      Logger.error("Ranger Admin service is not reachable, please restart the service and then try again")
+      return '',''

http://git-wip-us.apache.org/repos/asf/ambari/blob/260edad6/ambari-common/src/main/python/resource_management/libraries/functions/setup_ranger_plugin.py
----------------------------------------------------------------------
diff --git a/ambari-common/src/main/python/resource_management/libraries/functions/setup_ranger_plugin.py b/ambari-common/src/main/python/resource_management/libraries/functions/setup_ranger_plugin.py
index da5c559..eb22926 100644
--- a/ambari-common/src/main/python/resource_management/libraries/functions/setup_ranger_plugin.py
+++ b/ambari-common/src/main/python/resource_management/libraries/functions/setup_ranger_plugin.py
@@ -29,6 +29,7 @@ from resource_management.core.logger import Logger
 from resource_management.core.source import DownloadSource
 from resource_management.libraries.resources import ModifyPropertiesFile
 from resource_management.core.exceptions import Fail
+from resource_management.libraries.functions.ranger_functions_v2 import RangeradminV2
 
 def setup_ranger_plugin(component_select_name, service_name,
                         downloaded_custom_connector, driver_curl_source, 
@@ -36,7 +37,7 @@ def setup_ranger_plugin(component_select_name, service_name,
                         repo_name, plugin_repo_dict, 
                         ranger_env_properties, plugin_properties,
                         policy_user, policymgr_mgr_url,
-                        plugin_enabled):
+                        plugin_enabled,api_version=None):
   File(downloaded_custom_connector,
        content = DownloadSource(driver_curl_source)
   )
@@ -58,8 +59,11 @@ def setup_ranger_plugin(component_select_name, service_name,
 
   if plugin_enabled:
     cmd = (format('enable-{service_name}-plugin.sh'),)
-    
-    ranger_adm_obj = Rangeradmin(url=policymgr_mgr_url)
+    if api_version == 'v2' and api_version is not None:
+      ranger_adm_obj = RangeradminV2(url=policymgr_mgr_url)
+    else:
+      ranger_adm_obj = Rangeradmin(url=policymgr_mgr_url)
+
     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'], 
@@ -73,4 +77,4 @@ def setup_ranger_plugin(component_select_name, service_name,
         environment=cmd_env, 
         logoutput=True,
         sudo=True,
-  )                    
\ No newline at end of file
+  )                    

http://git-wip-us.apache.org/repos/asf/ambari/blob/260edad6/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration/yarn-env.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration/yarn-env.xml b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration/yarn-env.xml
index 1c23ade..c16c3ac 100644
--- a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration/yarn-env.xml
+++ b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/configuration/yarn-env.xml
@@ -57,7 +57,11 @@
     <value>1000</value>
     <description>Set to 0 to disallow root from submitting jobs. Set to 1000 to disallow all superusers from submitting jobs</description>
   </property>  
-
+  <property>
+    <name>is_supported_yarn_ranger</name>
+    <value>false</value>
+    <description>Set to false by default,  needs to be set to true in stacks that use Ranger Yarn Plugin</description>
+  </property>
   <!-- yarn-env.sh -->
   <property>
     <name>content</name>

http://git-wip-us.apache.org/repos/asf/ambari/blob/260edad6/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 004c786..ae736de 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
@@ -249,4 +249,73 @@ min_user_id = config['configurations']['yarn-env']['min_user_id']
 node_labels_dir = default("/configurations/yarn-site/yarn.node-labels.fs-store.root-dir", None)
 node_label_enable = config['configurations']['yarn-site']['yarn.node-labels.enabled']
 
-cgroups_dir = "/cgroups_test/cpu"
\ No newline at end of file
+cgroups_dir = "/cgroups_test/cpu"
+
+# ***********************  RANGER PLUGIN CHANGES ***********************
+# ranger host
+ranger_admin_hosts = default("/clusterHostInfo/ranger_admin_hosts", [])
+has_ranger_admin = not len(ranger_admin_hosts) == 0
+ambari_server_hostname = config['clusterHostInfo']['ambari_server_host'][0]
+# hostname of the active HDFS HA Namenode (only used when HA is enabled)
+dfs_ha_namenode_active = default("/configurations/hadoop-env/dfs_ha_initial_namenode_active", None)
+if dfs_ha_namenode_active is not None: 
+  namenode_hostname = dfs_ha_namenode_active
+else:
+  namenode_hostname = config['clusterHostInfo']['namenode_host'][0]
+
+ranger_admin_log_dir = default("/configurations/ranger-env/ranger_admin_log_dir","/var/log/ranger/admin")
+is_supported_yarn_ranger = config['configurations']['yarn-env']['is_supported_yarn_ranger']
+
+#ranger yarn properties
+if has_ranger_admin:
+
+  enable_ranger_yarn = (config['configurations']['ranger-yarn-plugin-properties']['ranger-yarn-plugin-enabled'].lower() == 'yes')
+  policymgr_mgr_url = config['configurations']['admin-properties']['policymgr_external_url']
+  sql_connector_jar = config['configurations']['admin-properties']['SQL_CONNECTOR_JAR']
+  xa_audit_db_flavor = config['configurations']['admin-properties']['DB_FLAVOR']
+  xa_audit_db_name = config['configurations']['admin-properties']['audit_db_name']
+  xa_audit_db_user = config['configurations']['admin-properties']['audit_db_user']
+  xa_audit_db_password = config['configurations']['admin-properties']['audit_db_password']
+  xa_db_host = config['configurations']['admin-properties']['db_host']
+  repo_name = str(config['clusterName']) + '_yarn'
+
+  ranger_env = config['configurations']['ranger-env']
+  ranger_plugin_properties = config['configurations']['ranger-yarn-plugin-properties']
+  policy_user = config['configurations']['ranger-yarn-plugin-properties']['policy_user']
+  
+  ranger_plugin_config = {
+    'username' : config['configurations']['ranger-yarn-plugin-properties']['REPOSITORY_CONFIG_USERNAME'],
+    'password' : config['configurations']['ranger-yarn-plugin-properties']['REPOSITORY_CONFIG_PASSWORD'],
+    'yarn.url' : config['configurations']['yarn-site']['yarn.resourcemanager.webapp.address'],
+    'commonNameForCertificate' : config['configurations']['ranger-yarn-plugin-properties']['common.name.for.certificate']
+  }
+
+  yarn_ranger_plugin_repo = {
+    'isEnabled': 'true',
+    'configs': ranger_plugin_config,
+    'description': 'yarn repo',
+    'name': repo_name,
+    'repositoryType': 'yarn',
+    'type': 'yarn',
+    'assetType': '1'
+  }
+  #For curl command in ranger plugin to get db connector
+  jdk_location = config['hostLevelParams']['jdk_location']
+  java_share_dir = '/usr/share/java'
+  if xa_audit_db_flavor and xa_audit_db_flavor.lower() == 'mysql':
+    jdbc_symlink_name = "mysql-jdbc-driver.jar"
+    jdbc_jar_name = "mysql-connector-java.jar"
+  elif xa_audit_db_flavor and xa_audit_db_flavor.lower() == 'oracle':
+    jdbc_jar_name = "ojdbc6.jar"
+    jdbc_symlink_name = "oracle-jdbc-driver.jar"
+  elif xa_audit_db_flavor and xa_audit_db_flavor.lower() == 'postgres':
+    jdbc_jar_name = "postgresql.jar"
+    jdbc_symlink_name = "postgres-jdbc-driver.jar"
+  elif xa_audit_db_flavor and xa_audit_db_flavor.lower() == 'sqlserver':
+    jdbc_jar_name = "sqljdbc4.jar"
+    jdbc_symlink_name = "mssql-jdbc-driver.jar"
+
+  downloaded_custom_connector = format("{tmp_dir}/{jdbc_jar_name}")
+
+  driver_curl_source = format("{jdk_location}/{jdbc_symlink_name}")
+  driver_curl_target = format("{java_share_dir}/{jdbc_jar_name}")

http://git-wip-us.apache.org/repos/asf/ambari/blob/260edad6/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/resourcemanager.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/resourcemanager.py b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/resourcemanager.py
index 9fa2aae..9d5fb97 100644
--- a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/resourcemanager.py
+++ b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/resourcemanager.py
@@ -31,7 +31,7 @@ from yarn import yarn
 from service import service
 from ambari_commons import OSConst
 from ambari_commons.os_family_impl import OsFamilyImpl
-
+from setup_ranger_yarn import setup_ranger_yarn
 
 
 class Resourcemanager(Script):
@@ -102,6 +102,8 @@ class ResourcemanagerDefault(Resourcemanager):
 
     env.set_params(params)
     self.configure(env) # FOR SECURITY
+    if params.is_supported_yarn_ranger:
+      setup_ranger_yarn() #Ranger Yarn Plugin related calls 
     if params.hdp_stack_version != "" and compare_versions(params.hdp_stack_version, '2.1') == 0:
       install_tez_jars()
     else:

http://git-wip-us.apache.org/repos/asf/ambari/blob/260edad6/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
new file mode 100644
index 0000000..0e3ed98
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/scripts/setup_ranger_yarn.py
@@ -0,0 +1,36 @@
+#!/usr/bin/env python
+"""
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+Unless required by applicable law or agreed to in writing, software
+distributed under the License is distributed on an "AS IS" BASIS,
+WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+See the License for the specific language governing permissions and
+limitations under the License.
+
+"""
+from resource_management.core.logger import Logger
+from resource_management.libraries.functions.setup_ranger_plugin import setup_ranger_plugin
+
+def setup_ranger_yarn():
+  import params
+
+  if params.has_ranger_admin:
+    api_version = 'v2'
+    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,
+                        params.ranger_env, params.ranger_plugin_properties,
+                        params.policy_user, params.policymgr_mgr_url,
+                        params.enable_ranger_yarn, api_version)                 
+  else:
+    Logger.info('Ranger admin not installed')
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/260edad6/ambari-server/src/main/resources/stacks/HDP/2.3/services/YARN/configuration/ranger-yarn-plugin-properties.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.3/services/YARN/configuration/ranger-yarn-plugin-properties.xml b/ambari-server/src/main/resources/stacks/HDP/2.3/services/YARN/configuration/ranger-yarn-plugin-properties.xml
new file mode 100644
index 0000000..93035d1
--- /dev/null
+++ b/ambari-server/src/main/resources/stacks/HDP/2.3/services/YARN/configuration/ranger-yarn-plugin-properties.xml
@@ -0,0 +1,235 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+-->
+<configuration supports_final="true">
+
+  <property>
+    <name>POLICY_MGR_URL</name>
+    <value>{{policymgr_mgr_url}}</value>
+    <description></description>
+  </property> 
+
+  <property>
+    <name>SQL_CONNECTOR_JAR</name>
+    <value>{{sql_connector_jar}}</value>
+    <description></description>
+  </property> 
+
+  <property>
+    <name>REPOSITORY_NAME</name>
+    <value>{{repo_name}}</value>
+    <description></description>
+  </property> 
+
+  <property>
+    <name>XAAUDIT.DB.FLAVOUR</name>
+    <value>{{xa_audit_db_flavor}}</value>
+    <description></description>
+  </property> 
+
+  <property>
+    <name>XAAUDIT.DB.HOSTNAME</name>
+    <value>{{xa_db_host}}</value>
+    <description></description>
+  </property> 
+
+  <property>
+    <name>XAAUDIT.DB.DATABASE_NAME</name>
+    <value>{{xa_audit_db_name}}</value>
+    <description></description>
+  </property> 
+
+  <property>
+      <name>XAAUDIT.DB.USER_NAME</name>
+      <value>{{xa_audit_db_user}}</value>
+      <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.DB.PASSWORD</name>
+    <value>{{xa_audit_db_password}}</value>
+    <description></description>
+  </property>
+
+
+  <property>
+    <name>policy_user</name>
+    <value>ambari-qa</value>
+    <description>This user must be system user and also present at Ranger admin portal</description>
+  </property> 
+
+  <property>
+    <name>hadoop.rpc.protection</name>
+    <value>-</value>
+    <description>Used for repository creation on ranger admin</description>
+  </property>
+
+  <property>
+    <name>common.name.for.certificate</name>
+    <value>-</value>
+    <description>Used for repository creation on ranger admin</description>
+  </property>
+
+  <property>
+    <name>ranger-yarn-plugin-enabled</name>
+    <value>No</value>
+    <description>Enable ranger yarn plugin ?</description>
+  </property>
+
+  <property>
+    <name>REPOSITORY_CONFIG_USERNAME</name>
+    <value>yarn</value>
+    <description>Used for repository creation on ranger admin</description>
+  </property>
+
+  <property>
+    <name>REPOSITORY_CONFIG_PASSWORD</name>
+    <value>yarn</value>
+    <property-type>PASSWORD</property-type>
+    <description>Used for repository creation on ranger admin</description>
+  </property> 
+
+  <property>
+    <name>XAAUDIT.DB.IS_ENABLED</name>
+    <value>true</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.HDFS.IS_ENABLED</name>
+    <value>false</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.HDFS.DESTINATION_DIRECTORY</name>
+    <value>hdfs://{{namenode_hostname}}:8020/ranger/audit/%app-type%/%time:yyyyMMdd%</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.HDFS.LOCAL_BUFFER_DIRECTORY</name>
+    <value>{{ranger_admin_log_dir}}/hadoop/%app-type%/audit</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.HDFS.LOCAL_ARCHIVE_DIRECTORY</name>
+    <value>{{ranger_admin_log_dir}}/hadoop/%app-type%/audit/archive</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.HDFS.DESTINTATION_FILE</name>
+    <value>%hostname%-audit.log</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.HDFS.DESTINTATION_FLUSH_INTERVAL_SECONDS</name>
+    <value>900</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.HDFS.DESTINTATION_ROLLOVER_INTERVAL_SECONDS</name>
+    <value>86400</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.HDFS.DESTINTATION_OPEN_RETRY_INTERVAL_SECONDS</name>
+    <value>60</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.HDFS.LOCAL_BUFFER_FILE</name>
+    <value>%time:yyyyMMdd-HHmm.ss%.log</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.HDFS.LOCAL_BUFFER_FLUSH_INTERVAL_SECONDS</name>
+    <value>60</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.HDFS.LOCAL_BUFFER_ROLLOVER_INTERVAL_SECONDS</name>
+    <value>600</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.HDFS.LOCAL_ARCHIVE_MAX_FILE_COUNT</name>
+    <value>10</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.SOLR.IS_ENABLED</name>
+    <value>false</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.SOLR.MAX_QUEUE_SIZE</name>
+    <value>1</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.SOLR.MAX_FLUSH_INTERVAL_MS</name>
+    <value>1000</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>XAAUDIT.SOLR.SOLR_URL</name>
+    <value>http://localhost:6083/solr/ranger_audits</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>SSL_KEYSTORE_FILE_PATH</name>
+    <value>/etc/hadoop/conf/ranger-plugin-keystore.jks</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>SSL_KEYSTORE_PASSWORD</name>
+    <value>myKeyFilePassword</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>SSL_TRUSTSTORE_FILE_PATH</name>
+    <value>/etc/hadoop/conf/ranger-plugin-truststore.jks</value>
+    <description></description>
+  </property>
+
+  <property>
+    <name>SSL_TRUSTSTORE_PASSWORD</name>
+    <value>changeit</value>
+    <description></description>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/ambari/blob/260edad6/ambari-server/src/main/resources/stacks/HDP/2.3/services/YARN/configuration/yarn-env.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.3/services/YARN/configuration/yarn-env.xml b/ambari-server/src/main/resources/stacks/HDP/2.3/services/YARN/configuration/yarn-env.xml
new file mode 100644
index 0000000..063e605
--- /dev/null
+++ b/ambari-server/src/main/resources/stacks/HDP/2.3/services/YARN/configuration/yarn-env.xml
@@ -0,0 +1,29 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+-->
+<configuration supports_final="true">
+
+  <property>
+    <name>is_supported_yarn_ranger</name>
+    <value>true</value>
+    <description>Set to false by default,  needs to be set to true in stacks that use Ranger Yarn Plugin</description>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/ambari/blob/260edad6/ambari-server/src/test/python/stacks/2.0.6/configs/default.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.0.6/configs/default.json b/ambari-server/src/test/python/stacks/2.0.6/configs/default.json
index 636047e..c90d085 100644
--- a/ambari-server/src/test/python/stacks/2.0.6/configs/default.json
+++ b/ambari-server/src/test/python/stacks/2.0.6/configs/default.json
@@ -490,8 +490,9 @@
             "yarn_user": "yarn", 
             "resourcemanager_heapsize": "1024", 
             "yarn_log_dir_prefix": "/var/log/hadoop-yarn",
-            "min_user_id": "1000"
-        }, 
+            "min_user_id": "1000",
+            "is_supported_yarn_ranger": "false"
+        },
         "hadoop-env": {
             "namenode_opt_maxnewsize": "200m", 
             "hdfs_log_dir_prefix": "/var/log/hadoop",

http://git-wip-us.apache.org/repos/asf/ambari/blob/260edad6/ambari-server/src/test/python/stacks/2.0.6/configs/secured.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.0.6/configs/secured.json b/ambari-server/src/test/python/stacks/2.0.6/configs/secured.json
index 8319b9f..f000ef3 100644
--- a/ambari-server/src/test/python/stacks/2.0.6/configs/secured.json
+++ b/ambari-server/src/test/python/stacks/2.0.6/configs/secured.json
@@ -536,7 +536,8 @@
             "yarn_user": "yarn", 
             "resourcemanager_heapsize": "1024", 
             "yarn_log_dir_prefix": "/var/log/hadoop-yarn",
-            "min_user_id": "1000"
+            "min_user_id": "1000",
+            "is_supported_yarn_ranger": "false"
         },
         "cluster-env": {
             "security_enabled": "true",

http://git-wip-us.apache.org/repos/asf/ambari/blob/260edad6/ambari-server/src/test/python/stacks/2.1/configs/default.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.1/configs/default.json b/ambari-server/src/test/python/stacks/2.1/configs/default.json
index 9bd6a38..db3b858 100644
--- a/ambari-server/src/test/python/stacks/2.1/configs/default.json
+++ b/ambari-server/src/test/python/stacks/2.1/configs/default.json
@@ -574,8 +574,9 @@
             "yarn_heapsize": "1024", 
             "yarn_user": "yarn", 
             "resourcemanager_heapsize": "1024", 
-            "yarn_log_dir_prefix": "/var/log/hadoop-yarn"
-        }, 
+            "yarn_log_dir_prefix": "/var/log/hadoop-yarn",
+            "is_supported_yarn_ranger": "false"
+        },
         "hadoop-env": {
             "namenode_opt_maxnewsize": "200m", 
             "hdfs_log_dir_prefix": "/var/log/hadoop",

http://git-wip-us.apache.org/repos/asf/ambari/blob/260edad6/ambari-server/src/test/python/stacks/2.1/configs/secured.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.1/configs/secured.json b/ambari-server/src/test/python/stacks/2.1/configs/secured.json
index 5b2ce2c..86f0fe9 100644
--- a/ambari-server/src/test/python/stacks/2.1/configs/secured.json
+++ b/ambari-server/src/test/python/stacks/2.1/configs/secured.json
@@ -548,7 +548,8 @@
             "yarn_heapsize": "1024", 
             "yarn_user": "yarn", 
             "resourcemanager_heapsize": "1024", 
-            "yarn_log_dir_prefix": "/var/log/hadoop-yarn"
+            "yarn_log_dir_prefix": "/var/log/hadoop-yarn",
+            "is_supported_yarn_ranger": "false"
         },
         "cluster-env": {
             "security_enabled": "true",

http://git-wip-us.apache.org/repos/asf/ambari/blob/260edad6/ambari-server/src/test/python/stacks/2.3/configs/default.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.3/configs/default.json b/ambari-server/src/test/python/stacks/2.3/configs/default.json
index 0108142..44a8b41 100644
--- a/ambari-server/src/test/python/stacks/2.3/configs/default.json
+++ b/ambari-server/src/test/python/stacks/2.3/configs/default.json
@@ -208,6 +208,11 @@
                 "yarn.nodemanager.local-dirs": "true"
             }
         },
+        "yarn-site": {
+          "final": {
+            "is_supported_yarn_ranger": "true"
+          }
+        },
         "hdfs-site": {
             "final": {
                 "dfs.web.ugi": "true",