You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by yu...@apache.org on 2015/01/17 04:53:56 UTC

[2/2] ambari git commit: AMBARI-8949. Support Ranger installation via Ambari. (Gautam Borad via yusaku)

AMBARI-8949. Support Ranger installation via Ambari. (Gautam Borad via yusaku)


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

Branch: refs/heads/trunk
Commit: 968c4b9de36f0c19f7c285dbee8770ad81ae6b3c
Parents: 521e2e3
Author: Yusaku Sako <yu...@hortonworks.com>
Authored: Fri Jan 16 19:52:52 2015 -0800
Committer: Yusaku Sako <yu...@hortonworks.com>
Committed: Fri Jan 16 19:52:52 2015 -0800

----------------------------------------------------------------------
 .../libraries/functions/ranger_functions.py     | 221 +++++++++++++++++++
 .../HBASE/0.96.0.2.0/metainfo.xml               |   1 +
 .../0.96.0.2.0/package/scripts/hbase_master.py  |   5 +-
 .../HBASE/0.96.0.2.0/package/scripts/params.py  |  13 ++
 .../package/scripts/setup_ranger_hbase.py       | 187 ++++++++++++++++
 .../common-services/HDFS/2.1.0.2.0/metainfo.xml |   1 +
 .../HDFS/2.1.0.2.0/package/scripts/namenode.py  |   4 +-
 .../HDFS/2.1.0.2.0/package/scripts/params.py    |  13 ++
 .../package/scripts/setup_ranger_hdfs.py        | 191 ++++++++++++++++
 .../HIVE/0.12.0.2.0/metainfo.xml                |   1 +
 .../0.12.0.2.0/package/scripts/hive_server.py   |   4 +-
 .../HIVE/0.12.0.2.0/package/scripts/params.py   |  12 +
 .../package/scripts/setup_ranger_hive.py        | 182 +++++++++++++++
 .../common-services/KNOX/0.5.0.2.2/metainfo.xml |   1 +
 .../0.5.0.2.2/package/scripts/knox_gateway.py   |   3 +
 .../KNOX/0.5.0.2.2/package/scripts/params.py    |  12 +
 .../package/scripts/setup_ranger_knox.py        | 184 +++++++++++++++
 .../0.4.0/configuration/admin-properties.xml    | 180 +++++++++++++++
 .../RANGER/0.4.0/configuration/ranger-env.xml   |  49 ++++
 .../0.4.0/configuration/usersync-properties.xml | 103 +++++++++
 .../common-services/RANGER/0.4.0/metainfo.xml   |  84 +++++++
 .../RANGER/0.4.0/package/scripts/params.py      |  43 ++++
 .../0.4.0/package/scripts/ranger_admin.py       |  58 +++++
 .../0.4.0/package/scripts/ranger_usersync.py    |  56 +++++
 .../0.4.0/package/scripts/service_check.py      |  53 +++++
 .../0.4.0/package/scripts/setup_ranger.py       | 153 +++++++++++++
 .../stacks/HDP/2.2/role_command_order.json      |  15 ++
 .../ranger-hbase-plugin-properties.xml          | 150 +++++++++++++
 .../ranger-hdfs-plugin-properties.xml           | 156 +++++++++++++
 .../ranger-hive-plugin-properties.xml           | 163 ++++++++++++++
 .../stacks/HDP/2.2/services/RANGER/metainfo.xml |  29 +++
 .../stacks/HDP/2.2/services/stack_advisor.py    | 126 ++++++++++-
 .../stacks/2.0.6/configs/client-upgrade.json    |   6 +
 .../python/stacks/2.0.6/configs/default.json    |   6 +
 .../stacks/2.0.6/configs/default_client.json    |   3 +
 .../python/stacks/2.0.6/configs/ha_default.json |   3 +
 .../python/stacks/2.0.6/configs/hbase-2.2.json  |   3 +
 .../stacks/2.0.6/configs/hbase-check-2.2.json   |   3 +
 .../stacks/2.0.6/configs/hbase-preupgrade.json  |   3 +
 .../stacks/2.0.6/configs/hbase-rs-2.2.json      |   3 +
 .../python/stacks/2.0.6/configs/secured.json    |   6 +
 .../stacks/2.0.6/configs/secured_client.json    |   3 +
 .../stacks/2.2/common/test_stack_advisor.py     | 163 ++++++++++++--
 .../test/python/stacks/2.2/configs/default.json |  10 +-
 .../python/stacks/2.2/configs/hive-upgrade.json |   3 +
 ambari-web/app/config.js                        |   1 +
 46 files changed, 2639 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-common/src/main/python/resource_management/libraries/functions/ranger_functions.py
----------------------------------------------------------------------
diff --git a/ambari-common/src/main/python/resource_management/libraries/functions/ranger_functions.py b/ambari-common/src/main/python/resource_management/libraries/functions/ranger_functions.py
new file mode 100644
index 0000000..4036611
--- /dev/null
+++ b/ambari-common/src/main/python/resource_management/libraries/functions/ranger_functions.py
@@ -0,0 +1,221 @@
+#!/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
+
+class Rangeradmin:
+  sInstance = None
+  def __init__(self, url= 'http://localhost:6080'):
+    
+    self.baseUrl      =  url 
+    self.urlLogin     = self.baseUrl + '/login.jsp'
+    self.urlLoginPost = self.baseUrl + '/j_spring_security_check'
+    self.urlRepos     = self.baseUrl + '/service/assets/assets'
+    self.urlReposPub  = self.baseUrl + '/service/public/api/repository'
+    self.urlPolicies  = self.baseUrl + '/service/public/api/policy'
+    self.urlGroups    = self.baseUrl + '/service/xusers/groups'
+    self.urlUsers     = self.baseUrl + '/service/xusers/users'   
+    self.urlSecUsers  = self.baseUrl + '/service/xusers/secure/users'   
+
+    self.session    = None
+    self.isLoggedIn = False
+
+  def get_repository_by_name_urllib2(self, name, component, status, usernamepassword):
+    try:
+      searchRepoURL = self.urlReposPub + "?name=" + name + "&type=" + component + "&status=" + status
+      request = urllib2.Request(searchRepoURL)
+      base64string = base64.encodestring(usernamepassword).replace('\n', '')
+      request.add_header("Content-Type", "application/json")   
+      request.add_header("Accept", "application/json")  
+      request.add_header("Authorization", "Basic %s" % base64string)   
+      result = urllib2.urlopen(request)
+      response_code =  result.getcode()
+      response = json.loads(result.read())
+
+      if response_code == 200 and len(response['vXRepositories']) > 0:
+        for repo in response['vXRepositories']:
+          repoDump = json.loads(json.JSONEncoder().encode(repo))
+          if repoDump['name'] == name:
+            return repoDump
+        return None
+      else:
+        return None
+    except urllib2.URLError, e:
+      if isinstance(e, urllib2.HTTPError):
+          Logger.error("HTTP Code: %s" % e.code)
+          Logger.error("HTTP Data: %s" % e.read())
+      else:
+          Logger.error("Error : %s" % (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_repository_urllib2(self, data, usernamepassword):
+    try:
+      searchRepoURL = self.urlReposPub
+      base64string = base64.encodestring('%s' % (usernamepassword)).replace('\n', '')
+      headers = {
+        'Accept': 'application/json',
+        "Content-Type": "application/json"
+      }
+      request = urllib2.Request(searchRepoURL, data, headers)
+      request.add_header("Authorization", "Basic %s" % base64string)   
+      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')
+        #Get Policies 
+        repoData     = json.loads(data)
+        repoName     = repoData['name']
+        typeOfPolicy = repoData['repositoryType']
+        ##Get Policies by repo name
+        policyList = self.get_policy_by_repo_name(name=repoName, component=typeOfPolicy, status="true", usernamepassword=usernamepassword)
+        if (len(policyList)) > 0 : 
+          policiesUpdateCount = 0
+          for policy in policyList:
+            updatedPolicyObj = self.get_policy_params(typeOfPolicy,policy)
+            policyResCode, policyResponse = self.update_ranger_policy(updatedPolicyObj['id'], json.dumps(updatedPolicyObj), usernamepassword)
+            if policyResCode == 200:
+              policiesUpdateCount = policiesUpdateCount+1
+            else:
+              Logger.info('Policy Update failed')  
+          ##Check for count of updated policies
+          if len(policyList) == policiesUpdateCount:
+            Logger.info("Ranger Repository created successfully and policies updated successfully providing ambari-qa user all permissions")
+            return response
+          else:
+            return None
+        else:
+          Logger.info("Policies not found for the newly created Repository")
+        return  None
+      else:
+        Logger.info('Repository creation failed')
+        return None  
+    except urllib2.URLError, e:
+      if isinstance(e, urllib2.HTTPError):
+          Logger.error("HTTP Code: %s" % e.code)
+          Logger.error("HTTP Data: %s" % e.read())
+      else:
+          Logger.error("Error: %s" % (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 ):
+    try:
+      request = urllib2.Request(url)
+      base64string = base64.encodestring(usernamepassword).replace('\n', '')
+      request.add_header("Content-Type", "application/json")   
+      request.add_header("Accept", "application/json")  
+      request.add_header("Authorization", "Basic %s" % base64string)   
+      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: %s" % e.code)
+          Logger.error("HTTP Data: %s" % e.read())
+      else:
+          Logger.error("Error : %s" % (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 get_policy_by_repo_name(self, name, component, status, usernamepassword):
+    try:
+      searchPolicyURL = self.urlPolicies + "?repositoryName=" + name + "&repositoryType=" + component + "&isEnabled=" + status
+      request = urllib2.Request(searchPolicyURL)
+      base64string = base64.encodestring(usernamepassword).replace('\n', '')
+      request.add_header("Content-Type", "application/json")   
+      request.add_header("Accept", "application/json")  
+      request.add_header("Authorization", "Basic %s" % base64string)   
+      result = urllib2.urlopen(request)
+      response_code =  result.getcode()
+      response = json.loads(result.read())
+      if response_code == 200 and len(response['vXPolicies']) > 0:
+          return response['vXPolicies']
+      else:
+        return None
+    except urllib2.URLError, e:
+      if isinstance(e, urllib2.HTTPError):
+          Logger.error("HTTP Code: %s" % e.code)
+          Logger.error("HTTP Data: %s" % e.read())
+      else:
+          Logger.error("Error: %s" % (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 update_ranger_policy(self, policyId, data, usernamepassword):
+    try:
+      searchRepoURL = self.urlPolicies +"/"+str(policyId)
+      base64string = base64.encodestring('%s' % (usernamepassword)).replace('\n', '')
+      headers = {
+        'Accept': 'application/json',
+        "Content-Type": "application/json"
+      }
+      request = urllib2.Request(searchRepoURL, data, headers)
+      request.add_header("Authorization", "Basic %s" % base64string)   
+      request.get_method = lambda: 'PUT'
+      result = urllib2.urlopen(request)
+      response_code =  result.getcode()
+      response = json.loads(json.JSONEncoder().encode(result.read()))
+      if response_code == 200 :
+        Logger.info('Policy updated Successfully')
+        return response_code, response
+      else:
+        Logger.error('Update Policy failed')
+        return None, None
+    except urllib2.URLError, e:
+      if isinstance(e, urllib2.HTTPError):
+          Logger.error("HTTP Code: %s" % e.code)
+          Logger.error("HTTP Data: %s" % e.read())
+      else:
+          Logger.error("Error: %s" % (e.reason))
+      return None, None
+    except httplib.BadStatusLine:
+      Logger.error("Ranger Admin service is not reachable, please restart the service and then try again")
+      return None, None
+
+  def get_policy_params(self, typeOfPolicy,policyObj): 
+    
+    typeOfPolicy = typeOfPolicy.lower()
+    if typeOfPolicy == "hdfs":
+      policyObj['permMapList'] = [{'userList':['ambari-qa'],'permList':  ['Read','Write', 'Execute', 'Admin']}]
+    elif typeOfPolicy == "hive":
+      policyObj['permMapList'] = [{'userList':['ambari-qa'], 'permList':[ 'Select','Update', 'Create', 'Drop', 'Alter', 'Index', 'Lock', 'All', 'Admin' ]}]
+    elif typeOfPolicy == "hbase":
+      policyObj['permMapList'] = [{'userList':['ambari-qa'],'permList':[ 'Read', 'Write', 'Create', 'Admin']}]
+    elif typeOfPolicy == "knox":
+      policyObj['permMapList'] = [{'userList':['ambari-qa'], 'permList': ['Allow','Admin']}]
+    elif typeOfPolicy == "storm" : 
+      policyObj['permMapList'] = [{'userList':['ambari-qa'], 'permList':[ 'Submit Topology', 'File Upload', 'Get Nimbus Conf', 'Get Cluster Info', 'File Download', 'Kill Topology', 'Rebalance', 'Activate','Deactivate', 'Get Topology Conf', 'Get Topology', 'Get User Topology', 'Get Topology Info', 'Upload New Credential', 'Admin']}]
+    return policyObj
+

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/metainfo.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/metainfo.xml b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/metainfo.xml
index 18572ef..c00889e 100644
--- a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/metainfo.xml
+++ b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/metainfo.xml
@@ -137,6 +137,7 @@
         <config-type>hbase-site</config-type>
         <config-type>hbase-env</config-type>
         <config-type>hbase-log4j</config-type>
+        <config-type>ranger-hbase-plugin-properties</config-type>
       </configuration-dependencies>
 
     </service>

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/hbase_master.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/hbase_master.py b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/hbase_master.py
index c9da84a..36ae06d 100644
--- a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/hbase_master.py
+++ b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/hbase_master.py
@@ -27,7 +27,7 @@ from hbase import hbase
 from hbase_service import hbase_service
 from hbase_decommission import hbase_decommission
 import upgrade
-
+from setup_ranger_hbase import setup_ranger_hbase
          
 class HbaseMaster(Script):
 
@@ -36,6 +36,7 @@ class HbaseMaster(Script):
 
   def install(self, env):
     self.install_packages(env)
+    setup_ranger_hbase(env)
     
   def configure(self, env):
     import params
@@ -52,7 +53,7 @@ class HbaseMaster(Script):
     import params
     env.set_params(params)
     self.configure(env) # for security
-
+    setup_ranger_hbase(env)  
     hbase_service( 'master',
       action = 'start'
     )

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params.py b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params.py
index ace3901..555774f 100644
--- a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/params.py
@@ -162,3 +162,16 @@ if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
     region_mover = format("/usr/hdp/current/hbase-{role_root}/bin/region_mover.rb")
     region_drainer = format("/usr/hdp/current/hbase-{role_root}/bin/draining_servers.rb")
     hbase_cmd = format("/usr/hdp/current/hbase-{role_root}/bin/hbase")
+
+if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
+  # Setting Flag value for ranger hbase plugin
+  enable_ranger_hbase = False
+  user_input = config['configurations']['ranger-hbase-plugin-properties']['ranger-hbase-plugin-enabled']
+  if user_input.lower() == 'yes':
+    enable_ranger_hbase = True
+  elif user_input.lower() == 'no':
+    enable_ranger_hbase = False
+
+# ranger host
+ranger_admin_hosts = default("/clusterHostInfo/ranger_admin_hosts", [])
+has_ranger_admin = not len(ranger_admin_hosts) == 0    

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/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
new file mode 100644
index 0000000..6daa0bf
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/package/scripts/setup_ranger_hbase.py
@@ -0,0 +1,187 @@
+#!/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 sys
+import fileinput
+import subprocess
+import json
+import re
+from resource_management import *
+from resource_management.libraries.functions.ranger_functions import Rangeradmin
+from resource_management.core.logger import Logger
+
+def setup_ranger_hbase(env):
+    import params
+    env.set_params(params)
+
+    if params.has_ranger_admin:
+        try:
+            command = 'hdp-select status hbase-client'
+            return_code, hdp_output = shell.call(command, timeout=20)
+        except Exception, e:
+            Logger.error(str(e))
+            raise Fail('Unable to execute hdp-select command to retrieve the version.')
+
+        if return_code != 0:
+            raise Fail('Unable to determine the current version because of a non-zero return code of {0}'.format(str(return_code)))
+
+        hdp_version = re.sub('hbase-client - ', '', hdp_output)
+        match = re.match('[0-9]+.[0-9]+.[0-9]+.[0-9]+-[0-9]+', hdp_version)
+
+        if match is None:
+            raise Fail('Failed to get extracted version')
+
+        file_path = '/usr/hdp/'+ hdp_version +'/ranger-hbase-plugin/install.properties'
+
+        ranger_hbase_dict = ranger_hbase_properties(params)
+        hbase_repo_data = hbase_repo_properties(params)
+
+        write_properties_to_file(file_path, ranger_hbase_dict)
+
+        if params.enable_ranger_hbase:
+            cmd = format('cd /usr/hdp/{hdp_version}/ranger-hbase-plugin/ && sh enable-hbase-plugin.sh')
+            ranger_adm_obj = Rangeradmin(url=ranger_hbase_dict['POLICY_MGR_URL'])
+            response_code, response_recieved = ranger_adm_obj.check_ranger_login_urllib2(ranger_hbase_dict['POLICY_MGR_URL'] + '/login.jsp', 'test:test')
+
+            if response_code is not None and response_code == 200:
+                repo = ranger_adm_obj.get_repository_by_name_urllib2(ranger_hbase_dict['REPOSITORY_NAME'], 'hbase', 'true', 'admin:admin')
+
+                if repo and repo['name'] == ranger_hbase_dict['REPOSITORY_NAME']:
+                    Logger.info('Hbase Repository exist')
+                else:
+                    response = ranger_adm_obj.create_repository_urllib2(hbase_repo_data, 'admin:admin')
+                    if response is not None:
+                        Logger.info('Hbase Repository created in Ranger admin')
+                    else:
+                        Logger.info('Hbase Repository creation failed in Ranger admin')
+            else:
+                Logger.info('Ranger service is not started on given host')
+        else:
+            cmd = format('cd /usr/hdp/{hdp_version}/ranger-hbase-plugin/ && sh disable-hbase-plugin.sh')
+
+        Execute(cmd, environment={'JAVA_HOME': params.java64_home}, logoutput=True)                    
+    else:
+        Logger.info('Ranger admin not installed')
+
+
+def write_properties_to_file(file_path, value):
+    for key in value:
+      modify_config(file_path, key, value[key])
+
+
+def modify_config(filepath, variable, setting):
+    var_found = False
+    already_set = False
+    V=str(variable)
+    S=str(setting)
+    # use quotes if setting has spaces #
+    if ' ' in S:
+        S = '%s' % S
+
+    for line in fileinput.input(filepath, inplace = 1):
+        # process lines that look like config settings #
+        if not line.lstrip(' ').startswith('#') and '=' in line:
+            _infile_var = str(line.split('=')[0].rstrip(' '))
+            _infile_set = str(line.split('=')[1].lstrip(' ').rstrip())
+            # only change the first matching occurrence #
+            if var_found == False and _infile_var.rstrip(' ') == V:
+                var_found = True
+                # don't change it if it is already set #
+                if _infile_set.lstrip(' ') == S:
+                    already_set = True
+                else:
+                    line = "%s=%s\n" % (V, S)
+
+        sys.stdout.write(line)
+
+    # Append the variable if it wasn't found #
+    if not var_found:
+        with open(filepath, "a") as f:
+            f.write("%s=%s\n" % (V, S))
+    elif already_set == True:
+        pass
+    else:
+        pass
+
+    return
+
+def ranger_hbase_properties(params):
+    ranger_hbase_properties = dict()
+
+    ranger_hbase_properties['POLICY_MGR_URL']           = params.config['configurations']['admin-properties']['policymgr_external_url']
+    ranger_hbase_properties['SQL_CONNECTOR_JAR']        = params.config['configurations']['admin-properties']['SQL_CONNECTOR_JAR']
+    ranger_hbase_properties['XAAUDIT.DB.FLAVOUR']       = params.config['configurations']['admin-properties']['DB_FLAVOR']
+    ranger_hbase_properties['XAAUDIT.DB.DATABASE_NAME'] = params.config['configurations']['admin-properties']['audit_db_name']
+    ranger_hbase_properties['XAAUDIT.DB.USER_NAME']     = params.config['configurations']['admin-properties']['audit_db_user']
+    ranger_hbase_properties['XAAUDIT.DB.PASSWORD']      = params.config['configurations']['admin-properties']['audit_db_password']
+    ranger_hbase_properties['XAAUDIT.DB.HOSTNAME']      = params.config['configurations']['admin-properties']['db_host']
+    ranger_hbase_properties['REPOSITORY_NAME']          = params.config['clusterName'] + '_hbase'
+
+    ranger_hbase_properties['XAAUDIT.DB.IS_ENABLED']   = params.config['configurations']['ranger-hbase-plugin-properties']['XAAUDIT.DB.IS_ENABLED']
+
+    ranger_hbase_properties['XAAUDIT.HDFS.IS_ENABLED'] = params.config['configurations']['ranger-hbase-plugin-properties']['XAAUDIT.HDFS.IS_ENABLED']
+    ranger_hbase_properties['XAAUDIT.HDFS.DESTINATION_DIRECTORY'] = params.config['configurations']['ranger-hbase-plugin-properties']['XAAUDIT.HDFS.DESTINATION_DIRECTORY']
+    ranger_hbase_properties['XAAUDIT.HDFS.LOCAL_BUFFER_DIRECTORY'] = params.config['configurations']['ranger-hbase-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_DIRECTORY']
+    ranger_hbase_properties['XAAUDIT.HDFS.LOCAL_ARCHIVE_DIRECTORY'] = params.config['configurations']['ranger-hbase-plugin-properties']['XAAUDIT.HDFS.LOCAL_ARCHIVE_DIRECTORY']
+    ranger_hbase_properties['XAAUDIT.HDFS.DESTINTATION_FILE'] = params.config['configurations']['ranger-hbase-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_FILE']
+    ranger_hbase_properties['XAAUDIT.HDFS.DESTINTATION_FLUSH_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hbase-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_FLUSH_INTERVAL_SECONDS']
+    ranger_hbase_properties['XAAUDIT.HDFS.DESTINTATION_ROLLOVER_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hbase-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_ROLLOVER_INTERVAL_SECONDS']
+    ranger_hbase_properties['XAAUDIT.HDFS.DESTINTATION_OPEN_RETRY_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hbase-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_OPEN_RETRY_INTERVAL_SECONDS']
+    ranger_hbase_properties['XAAUDIT.HDFS.LOCAL_BUFFER_FILE'] = params.config['configurations']['ranger-hbase-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_FILE']
+    ranger_hbase_properties['XAAUDIT.HDFS.LOCAL_BUFFER_FLUSH_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hbase-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_FLUSH_INTERVAL_SECONDS']
+    ranger_hbase_properties['XAAUDIT.HDFS.LOCAL_BUFFER_ROLLOVER_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hbase-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_ROLLOVER_INTERVAL_SECONDS']
+    ranger_hbase_properties['XAAUDIT.HDFS.LOCAL_ARCHIVE_MAX_FILE_COUNT'] = params.config['configurations']['ranger-hbase-plugin-properties']['XAAUDIT.HDFS.LOCAL_ARCHIVE_MAX_FILE_COUNT']
+    
+
+    ranger_hbase_properties['SSL_KEYSTORE_FILE_PATH'] = params.config['configurations']['ranger-hbase-plugin-properties']['SSL_KEYSTORE_FILE_PATH']
+    ranger_hbase_properties['SSL_KEYSTORE_PASSWORD'] = params.config['configurations']['ranger-hbase-plugin-properties']['SSL_KEYSTORE_PASSWORD']
+    ranger_hbase_properties['SSL_TRUSTSTORE_FILE_PATH'] = params.config['configurations']['ranger-hbase-plugin-properties']['SSL_TRUSTSTORE_FILE_PATH']
+    ranger_hbase_properties['SSL_TRUSTSTORE_PASSWORD'] = params.config['configurations']['ranger-hbase-plugin-properties']['SSL_TRUSTSTORE_PASSWORD']
+    
+    ranger_hbase_properties['UPDATE_XAPOLICIES_ON_GRANT_REVOKE'] = params.config['configurations']['ranger-hbase-plugin-properties']['UPDATE_XAPOLICIES_ON_GRANT_REVOKE']
+
+    return ranger_hbase_properties    
+
+def hbase_repo_properties(params):
+
+    config_dict = dict()
+    config_dict['username'] = params.config['configurations']['ranger-hbase-plugin-properties']['REPOSITORY_CONFIG_USERNAME']
+    config_dict['password'] = params.config['configurations']['ranger-hbase-plugin-properties']['REPOSITORY_CONFIG_PASSWORD']
+    config_dict['hadoop.security.authentication'] = params.config['configurations']['core-site']['hadoop.security.authentication']
+    config_dict['hbase.security.authentication'] = params.config['configurations']['hbase-site']['hbase.security.authentication']
+    config_dict['hbase.zookeeper.property.clientPort'] = params.config['configurations']['hbase-site']['hbase.zookeeper.property.clientPort']
+    config_dict['hbase.zookeeper.quorum'] = params.config['configurations']['hbase-site']['hbase.zookeeper.quorum']
+    config_dict['zookeeper.znode.parent'] =  params.config['configurations']['hbase-site']['zookeeper.znode.parent']
+
+    if params.config['configurations']['cluster-env']['security_enabled']:
+        config_dict['hbase.master.kerberos.principal'] = params.config['configurations']['hbase-site']['hbase.master.kerberos.principal']
+    else:
+        config_dict['hbase.master.kerberos.principal'] = ''
+
+    repo= dict()
+    repo['isActive']                = "true"
+    repo['config']                  = json.dumps(config_dict)
+    repo['description']             = "hbase repo"
+    repo['name']                    = params.config['clusterName'] + "_hbase"
+    repo['repositoryType']          = "Hbase"
+    repo['assetType']               = '2'
+
+    data = json.dumps(repo)
+
+    return data

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/metainfo.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/metainfo.xml b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/metainfo.xml
index ab7c95c..3936043 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/metainfo.xml
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/metainfo.xml
@@ -228,6 +228,7 @@
         <config-type>hadoop-env</config-type>
         <config-type>hadoop-policy</config-type>
         <config-type>hdfs-log4j</config-type>
+        <config-type>ranger-hdfs-plugin-properties</config-type>
       </configuration-dependencies>
     </service>
   </services>

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/namenode.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/namenode.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/namenode.py
index 83f34bc..1398c5c 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/namenode.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/namenode.py
@@ -36,7 +36,7 @@ from hdfs_namenode import namenode
 from hdfs import hdfs
 import hdfs_rebalance
 from utils import failover_namenode
-
+from setup_ranger_hdfs import setup_ranger_hdfs
 
 class NameNode(Script):
 
@@ -50,6 +50,7 @@ class NameNode(Script):
     env.set_params(params)
     #TODO we need this for HA because of manual steps
     self.configure(env)
+    setup_ranger_hdfs(env)
 
   def prepare_rolling_upgrade(self, env):
     namenode_upgrade.prepare_rolling_upgrade()
@@ -70,6 +71,7 @@ class NameNode(Script):
 
     env.set_params(params)
     self.configure(env)
+    setup_ranger_hdfs(env)
     namenode(action="start", rolling_restart=rolling_restart, env=env)
 
   def post_rolling_restart(self, env):

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py
index a4a0d3d..eb315cc 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py
@@ -295,3 +295,16 @@ ttnode_heapsize = "1024m"
 dtnode_heapsize = config['configurations']['hadoop-env']['dtnode_heapsize']
 mapred_pid_dir_prefix = default("/configurations/mapred-env/mapred_pid_dir_prefix","/var/run/hadoop-mapreduce")
 mapred_log_dir_prefix = default("/configurations/mapred-env/mapred_log_dir_prefix","/var/log/hadoop-mapreduce")
+
+# ranger host
+ranger_admin_hosts = default("/clusterHostInfo/ranger_admin_hosts", [])
+has_ranger_admin = not len(ranger_admin_hosts) == 0
+
+if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
+    # setting flag value for ranger hdfs plugin
+    enable_ranger_hdfs = False
+    user_input = config['configurations']['ranger-hdfs-plugin-properties']['ranger-hdfs-plugin-enabled']
+    if  user_input.lower() == 'yes':
+      enable_ranger_hdfs = True
+    elif user_input.lower() == 'no':
+      enable_ranger_hdfs = False
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/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
new file mode 100644
index 0000000..d93ba65
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/setup_ranger_hdfs.py
@@ -0,0 +1,191 @@
+#!/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 sys
+import fileinput
+import subprocess
+import json
+import re
+from resource_management import *
+from resource_management.libraries.functions.ranger_functions import Rangeradmin
+from resource_management.core.logger import Logger
+
+def setup_ranger_hdfs(env):
+    import params
+    env.set_params(params)
+
+    if params.has_ranger_admin:
+        try:
+            command = 'hdp-select status hadoop-client'
+            return_code, hdp_output = shell.call(command, timeout=20)
+        except Exception, e:
+            Logger.error(str(e))
+            raise Fail('Unable to execute hdp-select command to retrieve the version.')
+
+        if return_code != 0:
+            raise Fail('Unable to determine the current version because of a non-zero return code of {0}'.format(str(return_code)))
+
+        hdp_version = re.sub('hadoop-client - ', '', hdp_output)
+        match = re.match('[0-9]+.[0-9]+.[0-9]+.[0-9]+-[0-9]+', hdp_version)
+
+        if match is None:
+            raise Fail('Failed to get extracted version')
+
+        file_path = '/usr/hdp/'+ hdp_version +'/ranger-hdfs-plugin/install.properties'
+
+        ranger_hdfs_dict = ranger_hdfs_properties(params)
+        hdfs_repo_data = hdfs_repo_properties(params)        
+
+        write_properties_to_file(file_path, ranger_hdfs_dict)
+
+        if params.enable_ranger_hdfs:            
+            cmd = format('cd /usr/hdp/{hdp_version}/ranger-hdfs-plugin/ && sh enable-hdfs-plugin.sh')
+            ranger_adm_obj = Rangeradmin(url=ranger_hdfs_dict['POLICY_MGR_URL'])
+            response_code, response_recieved = ranger_adm_obj.check_ranger_login_urllib2(ranger_hdfs_dict['POLICY_MGR_URL'] + '/login.jsp', 'test:test')
+
+            if response_code is not None and response_code == 200:
+                repo = ranger_adm_obj.get_repository_by_name_urllib2(ranger_hdfs_dict['REPOSITORY_NAME'], 'hdfs', 'true', 'admin:admin')
+
+                if repo and repo['name'] == ranger_hdfs_dict['REPOSITORY_NAME']:
+                    Logger.info('HDFS Repository exist')
+                else:
+                    response = ranger_adm_obj.create_repository_urllib2(hdfs_repo_data, 'admin:admin')
+                    if response is not None:
+                        Logger.info('HDFS Repository created in Ranger Admin')
+                    else:
+                        Logger.info('HDFS Repository creation failed in Ranger Admin')
+            else:
+                Logger.info('Ranger service is not started on given host')
+        else:
+            cmd = format('cd /usr/hdp/{hdp_version}/ranger-hdfs-plugin/ && sh disable-hdfs-plugin.sh')
+
+        Execute(cmd, environment={'JAVA_HOME': params.java_home}, logoutput=True)            
+    else:
+        Logger.info('Ranger admin not installed')
+
+
+def write_properties_to_file(file_path, value):
+    for key in value:
+      modify_config(file_path, key, value[key])
+
+
+def modify_config(filepath, variable, setting):
+    var_found = False
+    already_set = False
+    V=str(variable)
+    S=str(setting)
+    # use quotes if setting has spaces #
+    if ' ' in S:
+        S = '%s' % S
+
+    for line in fileinput.input(filepath, inplace = 1):
+        # process lines that look like config settings #
+        if not line.lstrip(' ').startswith('#') and '=' in line:
+            _infile_var = str(line.split('=')[0].rstrip(' '))
+            _infile_set = str(line.split('=')[1].lstrip(' ').rstrip())
+            # only change the first matching occurrence #
+            if var_found == False and _infile_var.rstrip(' ') == V:
+                var_found = True
+                # don't change it if it is already set #
+                if _infile_set.lstrip(' ') == S:
+                    already_set = True
+                else:
+                    line = "%s=%s\n" % (V, S)
+
+        sys.stdout.write(line)
+
+    # Append the variable if it wasn't found #
+    if not var_found:
+        with open(filepath, "a") as f:
+            f.write("%s=%s\n" % (V, S))
+    elif already_set == True:
+        pass
+    else:
+        pass
+
+    return
+
+def ranger_hdfs_properties(params):
+    ranger_hdfs_properties = dict()
+
+    ranger_hdfs_properties['POLICY_MGR_URL']           = params.config['configurations']['admin-properties']['policymgr_external_url']
+    ranger_hdfs_properties['SQL_CONNECTOR_JAR']        = params.config['configurations']['admin-properties']['SQL_CONNECTOR_JAR']
+    ranger_hdfs_properties['XAAUDIT.DB.FLAVOUR']       = params.config['configurations']['admin-properties']['DB_FLAVOR']
+    ranger_hdfs_properties['XAAUDIT.DB.DATABASE_NAME'] = params.config['configurations']['admin-properties']['audit_db_name']
+    ranger_hdfs_properties['XAAUDIT.DB.USER_NAME']     = params.config['configurations']['admin-properties']['audit_db_user']
+    ranger_hdfs_properties['XAAUDIT.DB.PASSWORD']      = params.config['configurations']['admin-properties']['audit_db_password']
+    ranger_hdfs_properties['XAAUDIT.DB.HOSTNAME']      = params.config['configurations']['admin-properties']['db_host']
+    ranger_hdfs_properties['REPOSITORY_NAME']          = params.config['clusterName'] + '_hadoop'
+
+    ranger_hdfs_properties['XAAUDIT.DB.IS_ENABLED']   = params.config['configurations']['ranger-hdfs-plugin-properties']['XAAUDIT.DB.IS_ENABLED']
+
+    ranger_hdfs_properties['XAAUDIT.HDFS.IS_ENABLED'] = params.config['configurations']['ranger-hdfs-plugin-properties']['XAAUDIT.HDFS.IS_ENABLED']
+    ranger_hdfs_properties['XAAUDIT.HDFS.DESTINATION_DIRECTORY'] = params.config['configurations']['ranger-hdfs-plugin-properties']['XAAUDIT.HDFS.DESTINATION_DIRECTORY']
+    ranger_hdfs_properties['XAAUDIT.HDFS.LOCAL_BUFFER_DIRECTORY'] = params.config['configurations']['ranger-hdfs-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_DIRECTORY']
+    ranger_hdfs_properties['XAAUDIT.HDFS.LOCAL_ARCHIVE_DIRECTORY'] = params.config['configurations']['ranger-hdfs-plugin-properties']['XAAUDIT.HDFS.LOCAL_ARCHIVE_DIRECTORY']
+    ranger_hdfs_properties['XAAUDIT.HDFS.DESTINTATION_FILE'] = params.config['configurations']['ranger-hdfs-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_FILE']
+    ranger_hdfs_properties['XAAUDIT.HDFS.DESTINTATION_FLUSH_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hdfs-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_FLUSH_INTERVAL_SECONDS']
+    ranger_hdfs_properties['XAAUDIT.HDFS.DESTINTATION_ROLLOVER_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hdfs-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_ROLLOVER_INTERVAL_SECONDS']
+    ranger_hdfs_properties['XAAUDIT.HDFS.DESTINTATION_OPEN_RETRY_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hdfs-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_OPEN_RETRY_INTERVAL_SECONDS']
+    ranger_hdfs_properties['XAAUDIT.HDFS.LOCAL_BUFFER_FILE'] = params.config['configurations']['ranger-hdfs-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_FILE']
+    ranger_hdfs_properties['XAAUDIT.HDFS.LOCAL_BUFFER_FLUSH_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hdfs-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_FLUSH_INTERVAL_SECONDS']
+    ranger_hdfs_properties['XAAUDIT.HDFS.LOCAL_BUFFER_ROLLOVER_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hdfs-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_ROLLOVER_INTERVAL_SECONDS']
+    ranger_hdfs_properties['XAAUDIT.HDFS.LOCAL_ARCHIVE_MAX_FILE_COUNT'] = params.config['configurations']['ranger-hdfs-plugin-properties']['XAAUDIT.HDFS.LOCAL_ARCHIVE_MAX_FILE_COUNT']
+    
+
+    ranger_hdfs_properties['SSL_KEYSTORE_FILE_PATH'] = params.config['configurations']['ranger-hdfs-plugin-properties']['SSL_KEYSTORE_FILE_PATH']
+    ranger_hdfs_properties['SSL_KEYSTORE_PASSWORD'] = params.config['configurations']['ranger-hdfs-plugin-properties']['SSL_KEYSTORE_PASSWORD']
+    ranger_hdfs_properties['SSL_TRUSTSTORE_FILE_PATH'] = params.config['configurations']['ranger-hdfs-plugin-properties']['SSL_TRUSTSTORE_FILE_PATH']
+    ranger_hdfs_properties['SSL_TRUSTSTORE_PASSWORD'] = params.config['configurations']['ranger-hdfs-plugin-properties']['SSL_TRUSTSTORE_PASSWORD']
+
+    return ranger_hdfs_properties
+
+
+def hdfs_repo_properties(params):
+
+    config_dict = dict()
+    config_dict['username'] = params.config['configurations']['ranger-hdfs-plugin-properties']['REPOSITORY_CONFIG_USERNAME']
+    config_dict['password'] = params.config['configurations']['ranger-hdfs-plugin-properties']['REPOSITORY_CONFIG_PASSWORD']
+    config_dict['hadoop.security.authentication'] = params.config['configurations']['core-site']['hadoop.security.authentication']
+    config_dict['hadoop.security.authorization'] = params.config['configurations']['core-site']['hadoop.security.authorization']
+    config_dict['fs.default.name'] = params.config['configurations']['core-site']['fs.defaultFS']
+    config_dict['hadoop.security.auth_to_local'] = params.config['configurations']['core-site']['hadoop.security.auth_to_local']
+    config_dict['hadoop.rpc.protection'] = params.config['configurations']['ranger-hdfs-plugin-properties']['hadoop.rpc.protection']
+    config_dict['commonNameForCertificate'] = params.config['configurations']['ranger-hdfs-plugin-properties']['common.name.for.certificate']
+
+    if params.config['configurations']['cluster-env']['security_enabled']:
+        config_dict['dfs.datanode.kerberos.principal'] = params.config['configurations']['hdfs-site']['dfs.datanode.kerberos.principal']
+        config_dict['dfs.namenode.kerberos.principal'] = params.config['configurations']['hdfs-site']['dfs.namenode.kerberos.principal']
+        config_dict['dfs.secondary.namenode.kerberos.principal'] = params.config['configurations']['hdfs-site']['dfs.secondary.namenode.kerberos.principal']
+    else:
+        config_dict['dfs.datanode.kerberos.principal'] = ''
+        config_dict['dfs.namenode.kerberos.principal'] = ''
+        config_dict['dfs.secondary.namenode.kerberos.principal'] = ''
+
+    repo= dict()
+    repo['isActive']                = "true"
+    repo['config']                  = json.dumps(config_dict)
+    repo['description']             = "hdfs repo"
+    repo['name']                    = params.config['clusterName'] + "_hadoop"
+    repo['repositoryType']          = "Hdfs"
+    repo['assetType']               = '1'
+
+    data = json.dumps(repo)
+
+    return data

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/metainfo.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/metainfo.xml b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/metainfo.xml
index db48936..da662ce 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/metainfo.xml
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/metainfo.xml
@@ -288,6 +288,7 @@
         <config-type>hive-env</config-type>
         <config-type>webhcat-site</config-type>
         <config-type>webhcat-env</config-type>
+        <config-type>ranger-hive-plugin-properties</config-type>
       </configuration-dependencies>
     </service>
   </services>

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_server.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_server.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_server.py
index 906cc24..2fea051 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_server.py
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/hive_server.py
@@ -27,6 +27,7 @@ from resource_management.libraries.functions.security_commons import build_expec
   cached_kinit_executor, get_params_from_filesystem, validate_security_config_properties, \
   FILE_TYPE_XML
 from install_jars import install_tez_jars
+from setup_ranger_hive import setup_ranger_hive
 
 class HiveServer(Script):
 
@@ -36,6 +37,7 @@ class HiveServer(Script):
   def install(self, env):
     import params
     self.install_packages(env, exclude_packages=params.hive_exclude_packages)
+    setup_ranger_hive(env)
 
 
   def configure(self, env):
@@ -55,7 +57,7 @@ class HiveServer(Script):
     # This function is needed in HDP 2.2, but it is safe to call in earlier versions.
     copy_tarballs_to_hdfs('mapreduce', 'hive-server2', params.tez_user, params.hdfs_user, params.user_group)
     copy_tarballs_to_hdfs('tez', 'hive-server2', params.tez_user, params.hdfs_user, params.user_group)
-
+    setup_ranger_hive(env)    
     hive_service( 'hiveserver2', action = 'start',
       rolling_restart=rolling_restart )
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params.py
index 00bada8..701fa84 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/params.py
@@ -319,3 +319,15 @@ HdfsDirectory = functools.partial(
   kinit_path_local = kinit_path_local,
   bin_dir = hadoop_bin_dir
 )
+
+# ranger host
+ranger_admin_hosts = default("/clusterHostInfo/ranger_admin_hosts", [])
+has_ranger_admin = not len(ranger_admin_hosts) == 0
+if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >=0:
+    # setting flag value for ranger hive plugin
+    enable_ranger_hive = False
+    user_input = config['configurations']['ranger-hive-plugin-properties']['ranger-hive-plugin-enabled']
+    if  user_input.lower() == 'yes':
+      enable_ranger_hive = True
+    elif user_input.lower() == 'no':
+      enable_ranger_hive = False
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/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
new file mode 100644
index 0000000..cc6a1d4
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/scripts/setup_ranger_hive.py
@@ -0,0 +1,182 @@
+#!/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 sys
+import fileinput
+import subprocess
+import json
+import re
+from resource_management import *
+from resource_management.libraries.functions.ranger_functions import Rangeradmin
+from resource_management.core.logger import Logger
+
+def setup_ranger_hive(env):
+    import params
+    env.set_params(params)
+
+    if params.has_ranger_admin:
+        try:
+            command = 'hdp-select status hive-server2'
+            return_code, hdp_output = shell.call(command, timeout=20)
+        except Exception, e:
+            Logger.error(str(e))
+            raise Fail('Unable to execute hdp-select command to retrieve the version.')
+
+        if return_code != 0:
+            raise Fail('Unable to determine the current version because of a non-zero return code of {0}'.format(str(return_code)))
+
+        hdp_version = re.sub('hive-server2 - ', '', hdp_output)
+        match = re.match('[0-9]+.[0-9]+.[0-9]+.[0-9]+-[0-9]+', hdp_version)
+
+        if match is None:
+            raise Fail('Failed to get extracted version')
+
+        file_path = '/usr/hdp/'+ hdp_version +'/ranger-hive-plugin/install.properties'
+
+        ranger_hive_dict = ranger_hive_properties(params)
+        hive_repo_data = hive_repo_properties(params)
+
+        write_properties_to_file(file_path, ranger_hive_dict)
+    
+        if params.enable_ranger_hive:
+            cmd = format('cd /usr/hdp/{hdp_version}/ranger-hive-plugin/ && sh enable-hive-plugin.sh')
+            ranger_adm_obj = Rangeradmin(url=ranger_hive_dict['POLICY_MGR_URL'])
+            response_code, response_recieved = ranger_adm_obj.check_ranger_login_urllib2(ranger_hive_dict['POLICY_MGR_URL'] + '/login.jsp', 'test:test')
+
+            if response_code is not None and response_code == 200:
+                repo = ranger_adm_obj.get_repository_by_name_urllib2(ranger_hive_dict['REPOSITORY_NAME'], 'hive', 'true', 'admin:admin')
+
+                if repo and repo['name'] ==  ranger_hive_dict['REPOSITORY_NAME']:
+                    Logger.info('Hive Repository exist')
+                else:
+                    response = ranger_adm_obj.create_repository_urllib2(hive_repo_data, 'admin:admin')
+                    if response is not None:
+                        Logger.info('Hive Repository created in Ranger Admin')
+                    else:
+                        Logger.info('Hive Repository creation failed in Ranger Admin')
+            else:                        
+                Logger.info('Ranger service is not started on given host')
+        else:
+            cmd = format('cd /usr/hdp/{hdp_version}/ranger-hive-plugin/ && sh disable-hive-plugin.sh')
+        
+        Execute(cmd, environment={'JAVA_HOME': params.java64_home}, logoutput=True)
+    else:
+        Logger.info('Ranger admin not installed')
+
+
+def write_properties_to_file(file_path, value):
+    for key in value:
+      modify_config(file_path, key, value[key])
+
+
+def modify_config(filepath, variable, setting):
+    var_found = False
+    already_set = False
+    V=str(variable)
+    S=str(setting)
+    # use quotes if setting has spaces #
+    if ' ' in S:
+        S = '%s' % S
+
+    for line in fileinput.input(filepath, inplace = 1):
+        # process lines that look like config settings #
+        if not line.lstrip(' ').startswith('#') and '=' in line:
+            _infile_var = str(line.split('=')[0].rstrip(' '))
+            _infile_set = str(line.split('=')[1].lstrip(' ').rstrip())
+            # only change the first matching occurrence #
+            if var_found == False and _infile_var.rstrip(' ') == V:
+                var_found = True
+                # don't change it if it is already set #
+                if _infile_set.lstrip(' ') == S:
+                    already_set = True
+                else:
+                    line = "%s=%s\n" % (V, S)
+
+        sys.stdout.write(line)
+
+    # Append the variable if it wasn't found #
+    if not var_found:
+        with open(filepath, "a") as f:
+            f.write("%s=%s\n" % (V, S))
+    elif already_set == True:
+        pass
+    else:
+        pass
+
+    return
+
+def ranger_hive_properties(params):
+    ranger_hive_properties = dict()
+
+    ranger_hive_properties['POLICY_MGR_URL']           = params.config['configurations']['admin-properties']['policymgr_external_url']
+    ranger_hive_properties['SQL_CONNECTOR_JAR']        = params.config['configurations']['admin-properties']['SQL_CONNECTOR_JAR']
+    ranger_hive_properties['XAAUDIT.DB.FLAVOUR']       = params.config['configurations']['admin-properties']['DB_FLAVOR']
+    ranger_hive_properties['XAAUDIT.DB.DATABASE_NAME'] = params.config['configurations']['admin-properties']['audit_db_name']
+    ranger_hive_properties['XAAUDIT.DB.USER_NAME']     = params.config['configurations']['admin-properties']['audit_db_user']
+    ranger_hive_properties['XAAUDIT.DB.PASSWORD']      = params.config['configurations']['admin-properties']['audit_db_password']
+    ranger_hive_properties['XAAUDIT.DB.HOSTNAME']      = params.config['configurations']['admin-properties']['db_host']
+    ranger_hive_properties['REPOSITORY_NAME']          = params.config['clusterName'] + '_hive'
+
+    ranger_hive_properties['XAAUDIT.DB.IS_ENABLED']   = params.config['configurations']['ranger-hive-plugin-properties']['XAAUDIT.DB.IS_ENABLED']
+
+    ranger_hive_properties['XAAUDIT.HDFS.IS_ENABLED'] = params.config['configurations']['ranger-hive-plugin-properties']['XAAUDIT.HDFS.IS_ENABLED']
+    ranger_hive_properties['XAAUDIT.HDFS.DESTINATION_DIRECTORY'] = params.config['configurations']['ranger-hive-plugin-properties']['XAAUDIT.HDFS.DESTINATION_DIRECTORY']
+    ranger_hive_properties['XAAUDIT.HDFS.LOCAL_BUFFER_DIRECTORY'] = params.config['configurations']['ranger-hive-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_DIRECTORY']
+    ranger_hive_properties['XAAUDIT.HDFS.LOCAL_ARCHIVE_DIRECTORY'] = params.config['configurations']['ranger-hive-plugin-properties']['XAAUDIT.HDFS.LOCAL_ARCHIVE_DIRECTORY']
+    ranger_hive_properties['XAAUDIT.HDFS.DESTINTATION_FILE'] = params.config['configurations']['ranger-hive-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_FILE']
+    ranger_hive_properties['XAAUDIT.HDFS.DESTINTATION_FLUSH_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hive-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_FLUSH_INTERVAL_SECONDS']
+    ranger_hive_properties['XAAUDIT.HDFS.DESTINTATION_ROLLOVER_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hive-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_ROLLOVER_INTERVAL_SECONDS']
+    ranger_hive_properties['XAAUDIT.HDFS.DESTINTATION_OPEN_RETRY_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hive-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_OPEN_RETRY_INTERVAL_SECONDS']
+    ranger_hive_properties['XAAUDIT.HDFS.LOCAL_BUFFER_FILE'] = params.config['configurations']['ranger-hive-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_FILE']
+    ranger_hive_properties['XAAUDIT.HDFS.LOCAL_BUFFER_FLUSH_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hive-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_FLUSH_INTERVAL_SECONDS']
+    ranger_hive_properties['XAAUDIT.HDFS.LOCAL_BUFFER_ROLLOVER_INTERVAL_SECONDS'] = params.config['configurations']['ranger-hive-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_ROLLOVER_INTERVAL_SECONDS']
+    ranger_hive_properties['XAAUDIT.HDFS.LOCAL_ARCHIVE_MAX_FILE_COUNT'] = params.config['configurations']['ranger-hive-plugin-properties']['XAAUDIT.HDFS.LOCAL_ARCHIVE_MAX_FILE_COUNT']
+    
+
+    ranger_hive_properties['SSL_KEYSTORE_FILE_PATH'] = params.config['configurations']['ranger-hive-plugin-properties']['SSL_KEYSTORE_FILE_PATH']
+    ranger_hive_properties['SSL_KEYSTORE_PASSWORD'] = params.config['configurations']['ranger-hive-plugin-properties']['SSL_KEYSTORE_PASSWORD']
+    ranger_hive_properties['SSL_TRUSTSTORE_FILE_PATH'] = params.config['configurations']['ranger-hive-plugin-properties']['SSL_TRUSTSTORE_FILE_PATH']
+    ranger_hive_properties['SSL_TRUSTSTORE_PASSWORD'] = params.config['configurations']['ranger-hive-plugin-properties']['SSL_TRUSTSTORE_PASSWORD']
+
+    ranger_hive_properties['UPDATE_XAPOLICIES_ON_GRANT_REVOKE'] = params.config['configurations']['ranger-hive-plugin-properties']['UPDATE_XAPOLICIES_ON_GRANT_REVOKE']
+
+    return ranger_hive_properties
+
+def hive_repo_properties(params):
+
+    hive_host = params.config['clusterHostInfo']['hive_server_host'][0]
+
+    config_dict = dict()
+    config_dict['username'] = params.config['configurations']['ranger-hive-plugin-properties']['REPOSITORY_CONFIG_USERNAME']
+    config_dict['password'] = params.config['configurations']['ranger-hive-plugin-properties']['REPOSITORY_CONFIG_PASSWORD']
+    config_dict['jdbc.driverClassName'] = params.config['configurations']['ranger-hive-plugin-properties']['jdbc.driverClassName']
+    config_dict['jdbc.url'] = 'jdbc:hive2://' + hive_host + ':10000'
+    config_dict['commonNameForCertificate'] = params.config['configurations']['ranger-hive-plugin-properties']['common.name.for.certificate']
+
+    repo= dict()
+    repo['isActive']                = "true"
+    repo['config']                  = json.dumps(config_dict)
+    repo['description']             = "hive repo"
+    repo['name']                    = params.config['clusterName'] + '_hive'
+    repo['repositoryType']          = "Hive"
+    repo['assetType']               = '3'
+
+    data = json.dumps(repo)
+
+    return data

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/metainfo.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/metainfo.xml b/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/metainfo.xml
index ce4448f..810d3ff 100644
--- a/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/metainfo.xml
+++ b/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/metainfo.xml
@@ -81,6 +81,7 @@
         <config-type>gateway-site</config-type>
         <config-type>gateway-log4j</config-type>
         <config-type>topology</config-type>
+        <config-type>ranger-knox-plugin-properties</config-type>
       </configuration-dependencies>
     </service>
   </services>

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/knox_gateway.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/knox_gateway.py b/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/knox_gateway.py
index 7e5fe13..08a9425 100644
--- a/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/knox_gateway.py
+++ b/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/knox_gateway.py
@@ -26,6 +26,7 @@ import upgrade
 
 from knox import knox
 from ldap import ldap
+from setup_ranger_knox import setup_ranger_knox
 
 class KnoxGateway(Script):
 
@@ -40,6 +41,7 @@ class KnoxGateway(Script):
     File(format('{knox_conf_dir}/topologies/sandbox.xml'),
          action = "delete",
     )
+    setup_ranger_knox(env)
 
   def configure(self, env):
     import params
@@ -62,6 +64,7 @@ class KnoxGateway(Script):
     self.configure(env)
     daemon_cmd = format('{knox_bin} start')
     no_op_test = format('ls {knox_pid_file} >/dev/null 2>&1 && ps -p `cat {knox_pid_file}` >/dev/null 2>&1')
+    setup_ranger_knox(env)
     Execute(daemon_cmd,
             user=params.knox_user,
             environment={'JAVA_HOME': params.java_home},

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params.py b/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params.py
index 938d1d7..8d97c2e 100644
--- a/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/params.py
@@ -145,4 +145,16 @@ if security_enabled:
   _hostname_lowercase = config['hostname'].lower()
   knox_principal_name = config['configurations']['knox-env']['knox_principal_name'].replace('_HOST',_hostname_lowercase)
 
+# ranger host
+ranger_admin_hosts = default("/clusterHostInfo/ranger_admin_hosts", [])
+has_ranger_admin = not len(ranger_admin_hosts) == 0
 
+if hdp_stack_version != "" and compare_versions(hdp_stack_version, '2.2') >= 0:
+    # Setting Flag value for ranger hbase plugin
+    enable_ranger_knox = False
+    user_input = config['configurations']['ranger-knox-plugin-properties']['ranger-knox-plugin-enabled']
+    if user_input.lower() == 'yes':
+      enable_ranger_knox = True
+    elif user_input.lower() == 'no':
+      enable_ranger_knox = False
+      
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/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
new file mode 100644
index 0000000..7e2cea0
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/KNOX/0.5.0.2.2/package/scripts/setup_ranger_knox.py
@@ -0,0 +1,184 @@
+#!/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 sys
+import fileinput
+import subprocess
+import json
+import re
+from resource_management import *
+from resource_management.libraries.functions.ranger_functions import Rangeradmin
+from resource_management.core.logger import Logger
+
+def setup_ranger_knox(env):
+    import params
+    env.set_params(params)
+
+    try:
+        command = 'hdp-select status knox-server'
+        return_code, hdp_output = shell.call(command, timeout=20)
+    except Exception, e:
+        Logger.error(str(e))
+        raise Fail('Unable to execute hdp-select command to retrieve the version.')
+
+    if return_code != 0:
+        raise Fail('Unable to determine the current version because of a non-zero return code of {0}'.format(str(return_code)))
+
+    hdp_version = re.sub('knox-server - ', '', hdp_output)
+    match = re.match('[0-9]+.[0-9]+.[0-9]+.[0-9]+-[0-9]+', hdp_version)
+
+    if match is None:
+        raise Fail('Failed to get extracted version')
+
+    file_path = '/usr/hdp/'+ hdp_version +'/ranger-knox-plugin/install.properties'
+
+    if params.has_ranger_admin:
+
+        ranger_knox_dict = ranger_knox_properties(params)
+        knox_repo_data = knox_repo_properties(params)       
+
+        write_properties_to_file(file_path, ranger_knox_dict)
+
+        if params.enable_ranger_knox:
+            cmd = format('cd /usr/hdp/{hdp_version}/ranger-knox-plugin/ && sh enable-knox-plugin.sh')
+            ranger_adm_obj = Rangeradmin(url=ranger_knox_dict['POLICY_MGR_URL'])
+            response_code, response_recieved = ranger_adm_obj.check_ranger_login_urllib2(ranger_knox_dict['POLICY_MGR_URL'] + '/login.jsp', 'test:test')
+
+            if response_code is not None and response_code == 200:
+                repo = ranger_adm_obj.get_repository_by_name_urllib2(ranger_knox_dict['REPOSITORY_NAME'], 'knox', 'true', 'admin:admin')
+
+                if repo and repo['name'] == ranger_knox_dict['REPOSITORY_NAME']:
+                    Logger.info('Knox Repository exist')
+                else:
+                    response = ranger_adm_obj.create_repository_urllib2(knox_repo_data, 'admin:admin')
+                    if response is not None:
+                        Logger.info('Knox Repository created in Ranger Admin')
+                    else:
+                        Logger.info('Knox Repository creation failed in Ranger Admin')
+            else:
+                Logger.info('Ranger service is not started on given host')
+        else:
+            cmd = format('cd /usr/hdp/{hdp_version}/ranger-knox-plugin/ && sh disable-knox-plugin.sh')
+
+        Execute(cmd, environment={'JAVA_HOME': params.java_home}, logoutput=True)
+    else:
+        Logger.info('Ranger admin not installed') 
+
+
+def write_properties_to_file(file_path, value):
+    for key in value:
+      modify_config(file_path, key, value[key])
+
+
+def modify_config(filepath, variable, setting):
+    var_found = False
+    already_set = False
+    V=str(variable)
+    S=str(setting)
+    # use quotes if setting has spaces #
+    if ' ' in S:
+        S = '%s' % S
+
+    for line in fileinput.input(filepath, inplace = 1):
+        # process lines that look like config settings #
+        if not line.lstrip(' ').startswith('#') and '=' in line:
+            _infile_var = str(line.split('=')[0].rstrip(' '))
+            _infile_set = str(line.split('=')[1].lstrip(' ').rstrip())
+            # only change the first matching occurrence #
+            if var_found == False and _infile_var.rstrip(' ') == V:
+                var_found = True
+                # don't change it if it is already set #
+                if _infile_set.lstrip(' ') == S:
+                    already_set = True
+                else:
+                    line = "%s=%s\n" % (V, S)
+
+        sys.stdout.write(line)
+
+    # Append the variable if it wasn't found #
+    if not var_found:
+        with open(filepath, "a") as f:
+            f.write("%s=%s\n" % (V, S))
+    elif already_set == True:
+        pass
+    else:
+        pass
+
+    return
+
+def ranger_knox_properties(params):
+    ranger_knox_properties = dict()
+
+    ranger_knox_properties['POLICY_MGR_URL']           = params.config['configurations']['admin-properties']['policymgr_external_url']
+    ranger_knox_properties['SQL_CONNECTOR_JAR']        = params.config['configurations']['admin-properties']['SQL_CONNECTOR_JAR']
+    ranger_knox_properties['XAAUDIT.DB.FLAVOUR']       = params.config['configurations']['admin-properties']['DB_FLAVOR']
+    ranger_knox_properties['XAAUDIT.DB.DATABASE_NAME'] = params.config['configurations']['admin-properties']['audit_db_name']
+    ranger_knox_properties['XAAUDIT.DB.USER_NAME']     = params.config['configurations']['admin-properties']['audit_db_user']
+    ranger_knox_properties['XAAUDIT.DB.PASSWORD']      = params.config['configurations']['admin-properties']['audit_db_password']
+    ranger_knox_properties['XAAUDIT.DB.HOSTNAME']      = params.config['configurations']['admin-properties']['db_host']
+    ranger_knox_properties['REPOSITORY_NAME']          = params.config['clusterName'] + '_knox'
+
+    ranger_knox_properties['KNOX_HOME'] = params.config['configurations']['ranger-knox-plugin-properties']['KNOX_HOME']
+
+    ranger_knox_properties['XAAUDIT.DB.IS_ENABLED']   = params.config['configurations']['ranger-knox-plugin-properties']['XAAUDIT.DB.IS_ENABLED']
+
+    ranger_knox_properties['XAAUDIT.HDFS.IS_ENABLED'] = params.config['configurations']['ranger-knox-plugin-properties']['XAAUDIT.HDFS.IS_ENABLED']
+    ranger_knox_properties['XAAUDIT.HDFS.DESTINATION_DIRECTORY'] = params.config['configurations']['ranger-knox-plugin-properties']['XAAUDIT.HDFS.DESTINATION_DIRECTORY']
+    ranger_knox_properties['XAAUDIT.HDFS.LOCAL_BUFFER_DIRECTORY'] = params.config['configurations']['ranger-knox-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_DIRECTORY']
+    ranger_knox_properties['XAAUDIT.HDFS.LOCAL_ARCHIVE_DIRECTORY'] = params.config['configurations']['ranger-knox-plugin-properties']['XAAUDIT.HDFS.LOCAL_ARCHIVE_DIRECTORY']
+    ranger_knox_properties['XAAUDIT.HDFS.DESTINTATION_FILE'] = params.config['configurations']['ranger-knox-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_FILE']
+    ranger_knox_properties['XAAUDIT.HDFS.DESTINTATION_FLUSH_INTERVAL_SECONDS'] = params.config['configurations']['ranger-knox-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_FLUSH_INTERVAL_SECONDS']
+    ranger_knox_properties['XAAUDIT.HDFS.DESTINTATION_ROLLOVER_INTERVAL_SECONDS'] = params.config['configurations']['ranger-knox-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_ROLLOVER_INTERVAL_SECONDS']
+    ranger_knox_properties['XAAUDIT.HDFS.DESTINTATION_OPEN_RETRY_INTERVAL_SECONDS'] = params.config['configurations']['ranger-knox-plugin-properties']['XAAUDIT.HDFS.DESTINTATION_OPEN_RETRY_INTERVAL_SECONDS']
+    ranger_knox_properties['XAAUDIT.HDFS.LOCAL_BUFFER_FILE'] = params.config['configurations']['ranger-knox-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_FILE']
+    ranger_knox_properties['XAAUDIT.HDFS.LOCAL_BUFFER_FLUSH_INTERVAL_SECONDS'] = params.config['configurations']['ranger-knox-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_FLUSH_INTERVAL_SECONDS']
+    ranger_knox_properties['XAAUDIT.HDFS.LOCAL_BUFFER_ROLLOVER_INTERVAL_SECONDS'] = params.config['configurations']['ranger-knox-plugin-properties']['XAAUDIT.HDFS.LOCAL_BUFFER_ROLLOVER_INTERVAL_SECONDS']
+    ranger_knox_properties['XAAUDIT.HDFS.LOCAL_ARCHIVE_MAX_FILE_COUNT'] = params.config['configurations']['ranger-knox-plugin-properties']['XAAUDIT.HDFS.LOCAL_ARCHIVE_MAX_FILE_COUNT']
+    
+
+    ranger_knox_properties['SSL_KEYSTORE_FILE_PATH'] = params.config['configurations']['ranger-knox-plugin-properties']['SSL_KEYSTORE_FILE_PATH']
+    ranger_knox_properties['SSL_KEYSTORE_PASSWORD'] = params.config['configurations']['ranger-knox-plugin-properties']['SSL_KEYSTORE_PASSWORD']
+    ranger_knox_properties['SSL_TRUSTSTORE_FILE_PATH'] = params.config['configurations']['ranger-knox-plugin-properties']['SSL_TRUSTSTORE_FILE_PATH']
+    ranger_knox_properties['SSL_TRUSTSTORE_PASSWORD'] = params.config['configurations']['ranger-knox-plugin-properties']['SSL_TRUSTSTORE_PASSWORD']
+    
+
+    return ranger_knox_properties    
+
+def knox_repo_properties(params):
+
+    knoxHost = params.config['clusterHostInfo']['knox_gateway_hosts'][0]
+    knoxPort = params.config['configurations']['gateway-site']['gateway.port']
+
+    config_dict = dict()
+    config_dict['username'] = params.config['configurations']['ranger-knox-plugin-properties']['REPOSITORY_CONFIG_USERNAME']
+    config_dict['password'] = params.config['configurations']['ranger-knox-plugin-properties']['REPOSITORY_CONFIG_USERNAME']
+    config_dict['knox.url'] = 'https://' + knoxHost + ':' + str(knoxPort) +'/gateway/admin/api/v1/topologies'
+    config_dict['commonNameForCertificate'] = params.config['configurations']['ranger-knox-plugin-properties']['common.name.for.certificate']
+
+    repo= dict()
+    repo['isActive']                = "true"
+    repo['config']                  = json.dumps(config_dict)
+    repo['description']             = "knox repo"
+    repo['name']                    = params.config['clusterName'] + "_knox"
+    repo['repositoryType']          = "Knox"
+    repo['assetType']               = '5'
+
+    data = json.dumps(repo)
+
+    return data

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/RANGER/0.4.0/configuration/admin-properties.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/RANGER/0.4.0/configuration/admin-properties.xml b/ambari-server/src/main/resources/common-services/RANGER/0.4.0/configuration/admin-properties.xml
new file mode 100644
index 0000000..687ea15
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/RANGER/0.4.0/configuration/admin-properties.xml
@@ -0,0 +1,180 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+/**
+ * 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="false">
+
+	<property>
+		<name>DB_FLAVOR</name>
+		<value>MYSQL</value>
+		<description>The database type to be used (mysql/oracle)</description>
+	</property>
+
+        <property>
+                <name>SQL_COMMAND_INVOKER</name>
+                <value>mysql</value>
+                <description>The executable path to be used to invoke command-line MYSQL</description>
+        </property>
+
+        <property>
+                <name>SQL_CONNECTOR_JAR</name>
+                <value>/usr/share/java/mysql-connector-java.jar</value>
+                <description>Location of DB client library (please check the location of the jar file)</description>
+        </property>
+
+        <property>
+                <name>db_root_user</name>
+                <value>root</value>
+                <property-type>USER</property-type>
+                <description>Database admin user</description>
+        </property>
+
+        <property require-input="true">
+                <name>db_root_password</name>
+                <value>vagrant</value>
+                <property-type>PASSWORD</property-type>
+                <description>Database password for the database admin user-id</description>
+        </property>
+
+        <property>
+                <name>db_host</name>
+                <value>localhost</value>
+                <description>Database host</description>
+        </property>
+
+        <property>
+                <name>db_name</name>
+                <value>ranger</value>
+                <description>Database name</description>
+        </property>
+
+        <property>
+                <name>db_user</name>
+                <value>rangeradmin</value>
+                <property-type>USER</property-type>
+                <description>Database user-id used for the XASecure schema</description>
+        </property>
+
+        <property require-input="true">
+                <name>db_password</name>
+                <value>rangeradmin</value>
+                <property-type>PASSWORD</property-type>
+                <description>Database password for the XASecure schema</description>
+        </property>
+
+        <property>
+                <name>audit_db_name</name>
+                <value>ranger_audit</value>
+                <description>Audit database name</description>
+        </property>
+
+        <property>
+                <name>audit_db_user</name>
+                <value>rangerlogger</value>
+                <property-type>USER</property-type>
+                <description>Database user-id for storing auditlog information</description>
+        </property>
+
+        <property require-input="true">
+                <name>audit_db_password</name>
+                <value>rangerlogger</value>
+                <property-type>PASSWORD</property-type>
+                <description>Database password for storing auditlog information</description>
+        </property>
+
+        <property>
+                <name>policymgr_external_url</name>
+                <value>http://localhost:6080</value>
+                <description>Policy Manager external url</description>
+        </property>
+
+        <property>
+                <name>policymgr_http_enabled</name>
+                <value>true</value>
+                <description>HTTP Enabled</description>
+        </property>
+
+        <property>
+                <name>authentication_method</name>
+                <value>UNIX</value>
+                <description></description>
+        </property>
+
+        <property>
+                <name>remoteLoginEnabled</name>
+                <value>true</value>
+                <description></description>
+        </property>
+
+        <property>
+                <name>authServiceHostName</name>
+                <value>localhost</value>
+                <description></description>
+        </property>
+
+        <property>
+                <name>authServicePort</name>
+                <value>5151</value>
+                <description></description>
+        </property>
+
+        <property>
+                <name>xa_ldap_url</name>
+                <value>"ldap://71.127.43.33:389"</value>
+                <description></description>
+        </property>
+
+        <property>
+                <name>xa_ldap_userDNpattern</name>
+                <value>"uid={0},ou=users,dc=xasecure,dc=net"</value>
+                <description></description>
+        </property>
+
+        <property>
+                <name>xa_ldap_groupSearchBase</name>
+                <value>"ou=groups,dc=xasecure,dc=net"</value>
+                <description></description>
+        </property>
+
+        <property>
+                <name>xa_ldap_groupSearchFilter</name>
+                <value>"(member=uid={0},ou=users,dc=xasecure,dc=net)"</value>
+                <description></description>
+        </property>
+
+        <property>
+                <name>xa_ldap_groupRoleAttribute</name>
+                <value>"cn"</value>
+                <description></description>
+        </property>
+
+        <property>
+                <name>xa_ldap_ad_domain</name>
+                <value>"xasecure.net"</value>
+                <description></description>
+        </property>
+
+        <property>
+                <name>xa_ldap_ad_url</name>
+                <value>"ldap://ad.xasecure.net:389"</value>
+                <description></description>
+        </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/RANGER/0.4.0/configuration/ranger-env.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/RANGER/0.4.0/configuration/ranger-env.xml b/ambari-server/src/main/resources/common-services/RANGER/0.4.0/configuration/ranger-env.xml
new file mode 100644
index 0000000..fbe5d64
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/RANGER/0.4.0/configuration/ranger-env.xml
@@ -0,0 +1,49 @@
+<?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>ranger_user</name>
+        <value>ranger</value>
+        <property-type>USER</property-type>
+        <description>Ranger username</description>
+    </property>
+
+    <property>
+        <name>ranger_group</name>
+        <value>ranger</value>
+        <property-type>GROUP</property-type>
+        <description>Ranger group</description>
+    </property>
+
+    <property>
+        <name>ranger_admin_log_dir</name>
+        <value>/var/log/ranger/admin</value>
+        <description></description>
+    </property>
+
+    <property>
+        <name>ranger_usersync_log_dir</name>
+        <value>/var/log/ranger/usersync</value>
+        <description></description>
+    </property>    
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/ambari/blob/968c4b9d/ambari-server/src/main/resources/common-services/RANGER/0.4.0/configuration/usersync-properties.xml
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/RANGER/0.4.0/configuration/usersync-properties.xml b/ambari-server/src/main/resources/common-services/RANGER/0.4.0/configuration/usersync-properties.xml
new file mode 100644
index 0000000..15838a2
--- /dev/null
+++ b/ambari-server/src/main/resources/common-services/RANGER/0.4.0/configuration/usersync-properties.xml
@@ -0,0 +1,103 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+/**
+ * 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="false">
+	<property>
+		<name>SYNC_SOURCE</name>
+		<value>unix</value>
+		<description></description>
+	</property>
+	<property>
+		<name>MIN_UNIX_USER_ID_TO_SYNC</name>
+		<value>1000</value>
+		<description></description>
+	</property>
+	<property>
+		<name>SYNC_INTERVAL</name>
+		<value>1</value>
+		<description></description>
+	</property>
+	<property>
+		<name>SYNC_LDAP_URL</name>
+		<value>ldap://localhost:389</value>
+		<description>a sample value would be:  ldap://ldap.example.com:389</description>
+	</property>
+	<property>
+		<name>SYNC_LDAP_BIND_DN</name>
+		<value>cn=admin,dc=xasecure,dc=net</value>
+		<description>a sample value would be cn=admin,ou=users,dc=hadoop,dc=apache,dc-org</description>
+	</property>
+	<property>
+		<name>SYNC_LDAP_BIND_PASSWORD</name>
+		<value>admin321</value>
+		<description></description>
+	</property>
+	<property>
+		<name>CRED_KEYSTORE_FILENAME</name>
+		<value>/usr/lib/xausersync/.jceks/xausersync.jceks</value>
+		<description></description>
+	</property>
+	<property>
+		<name>SYNC_LDAP_USER_SEARCH_BASE</name>
+		<value>ou=users,dc=xasecure,dc=net</value>
+		<description>sample value would be ou=users,dc=hadoop,dc=apache,dc=org</description>
+	</property>
+	<property>
+		<name>SYNC_LDAP_USER_SEARCH_SCOPE</name>
+		<value>sub</value>
+		<description>default value: sub</description>
+	</property>
+	<property>
+		<name>SYNC_LDAP_USER_OBJECT_CLASS</name>
+		<value>person</value>
+		<description>default value: person</description>
+	</property>
+	<property>
+		<name>SYNC_LDAP_USER_SEARCH_FILTER</name>
+		<value>-</value>
+		<description>default value is empty</description>
+	</property>
+	<property>
+		<name>SYNC_LDAP_USER_NAME_ATTRIBUTE</name>
+		<value>cn</value>
+		<description>default value: cn</description>
+	</property>
+	<property>
+		<name>SYNC_LDAP_USER_GROUP_NAME_ATTRIBUTE</name>
+		<value>memberof,ismemberof</value>
+		<description></description>
+	</property>
+	<property>
+		<name>SYNC_LDAP_USERNAME_CASE_CONVERSION</name>
+		<value>lower</value>
+		<description>possible values:  none, lower, upper</description>
+	</property>
+	<property>
+		<name>SYNC_LDAP_GROUPNAME_CASE_CONVERSION</name>
+		<value>lower</value>
+		<description>possible values:  none, lower, upper</description>
+	</property>
+	<property>
+		<name>logdir</name>
+		<value>logs</value>
+		<description>user sync log path</description>
+	</property>
+</configuration>