You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ranger.apache.org by me...@apache.org on 2020/09/04 17:16:06 UTC

[ranger] 03/03: RANGER-2929 :- Update Atlas Ranger Authorizer for "type-read" acessType changes done in ATLAS-3898.

This is an automated email from the ASF dual-hosted git repository.

mehul pushed a commit to branch ranger-2.2
in repository https://gitbox.apache.org/repos/asf/ranger.git

commit 03d4b3e1a053f0a79899051e1064213d2b66286a
Author: nixonrodrigues <ni...@apache.org>
AuthorDate: Mon Aug 31 12:56:19 2020 +0530

    RANGER-2929 :- Update Atlas Ranger Authorizer for "type-read" acessType changes done in ATLAS-3898.
    
    Signed-off-by: Mehul Parikh <me...@apache.org>
---
 .../service-defs/ranger-servicedef-atlas.json      |  25 ++-
 .../atlas/authorizer/RangerAtlasAuthorizer.java    |  48 +++-
 .../ranger/services/atlas/RangerServiceAtlas.java  |   9 +
 pom.xml                                            |   2 +-
 .../atlas/authorizer/RangerAtlasAuthorizer.java    |  25 +++
 .../optimized/current/ranger_core_db_mysql.sql     |   1 +
 .../optimized/current/ranger_core_db_oracle.sql    |   1 +
 .../optimized/current/ranger_core_db_postgres.sql  |   1 +
 .../current/ranger_core_db_sqlanywhere.sql         |   2 +
 .../optimized/current/ranger_core_db_sqlserver.sql |   1 +
 .../patch/PatchForAtlasToAddTypeRead_J10040.java   | 241 +++++++++++++++++++++
 11 files changed, 349 insertions(+), 7 deletions(-)

diff --git a/agents-common/src/main/resources/service-defs/ranger-servicedef-atlas.json b/agents-common/src/main/resources/service-defs/ranger-servicedef-atlas.json
index 7672be0..68a3d36 100644
--- a/agents-common/src/main/resources/service-defs/ranger-servicedef-atlas.json
+++ b/agents-common/src/main/resources/service-defs/ranger-servicedef-atlas.json
@@ -42,7 +42,7 @@
 			},
 			"label": "Type Name",
 			"description": "Type Name",
-			"accessTypeRestrictions": ["type-create", "type-update", "type-delete"]
+			"accessTypeRestrictions": ["type-read" ,"type-create", "type-update", "type-delete" ]
 		},
 		{
 			"itemId": 3,
@@ -298,17 +298,29 @@
 		{
 			"itemId": 1,
 			"name": "type-create",
-			"label": "Create Type"
+			"label": "Create Type",
+			"impliedGrants":
+			[
+				"type-read"
+			]
 		},
 		{
 			"itemId": 2,
 			"name": "type-update",
-			"label": "Update Type"
+			"label": "Update Type",
+			"impliedGrants":
+			[
+				"type-read"
+			]
 		},
 		{
 			"itemId": 3,
 			"name": "type-delete",
-			"label": "Delete Type"
+			"label": "Delete Type",
+			"impliedGrants":
+			[
+				"type-read"
+			]
 		},
 		{
 			"itemId": 4,
@@ -389,6 +401,11 @@
 			"itemId": 19,
 			"name": "entity-update-business-metadata",
 			"label": "Update Business Metadata"
+		},
+		{
+			"itemId": 20,
+			"name": "type-read",
+			"label": "Read Type"
 		}
 	],
 	"configs": [
diff --git a/plugin-atlas/src/main/java/org/apache/ranger/authorization/atlas/authorizer/RangerAtlasAuthorizer.java b/plugin-atlas/src/main/java/org/apache/ranger/authorization/atlas/authorizer/RangerAtlasAuthorizer.java
index 28d71de..79ef604 100644
--- a/plugin-atlas/src/main/java/org/apache/ranger/authorization/atlas/authorizer/RangerAtlasAuthorizer.java
+++ b/plugin-atlas/src/main/java/org/apache/ranger/authorization/atlas/authorizer/RangerAtlasAuthorizer.java
@@ -25,10 +25,13 @@ import org.apache.atlas.authorize.AtlasAuthorizationException;
 import org.apache.atlas.authorize.AtlasEntityAccessRequest;
 import org.apache.atlas.authorize.AtlasSearchResultScrubRequest;
 import org.apache.atlas.authorize.AtlasRelationshipAccessRequest;
-
+import org.apache.atlas.authorize.AtlasTypesDefFilterRequest;
 import org.apache.atlas.authorize.AtlasTypeAccessRequest;
+import org.apache.atlas.authorize.AtlasAccessRequest;
 import org.apache.atlas.authorize.AtlasAuthorizer;
 import org.apache.atlas.authorize.AtlasPrivilege;
+import org.apache.atlas.model.typedef.AtlasBaseTypeDef;
+import org.apache.atlas.model.typedef.AtlasTypesDef;
 import org.apache.atlas.model.discovery.AtlasSearchResult;
 import org.apache.atlas.model.instance.AtlasEntityHeader;
 import org.apache.commons.collections.CollectionUtils;
@@ -190,7 +193,14 @@ public class RangerAtlasAuthorizer implements AtlasAuthorizer {
             rangerRequest.setForwardedAddresses(request.getForwardedAddresses());
             rangerRequest.setRemoteIPAddress(request.getRemoteIPAddress());
 
-            ret = checkAccess(rangerRequest);
+            boolean isAuditDisabled = ACCESS_TYPE_TYPE_READ.equalsIgnoreCase(action);
+
+            if (isAuditDisabled) {
+                ret = checkAccess(rangerRequest, null);
+            } else {
+                ret = checkAccess(rangerRequest);
+            }
+
         } finally {
             RangerPerfTracer.log(perf);
         }
@@ -202,6 +212,8 @@ public class RangerAtlasAuthorizer implements AtlasAuthorizer {
         return ret;
     }
 
+
+
     public boolean isAccessAllowed(AtlasRelationshipAccessRequest request) throws AtlasAuthorizationException {
         if (LOG.isDebugEnabled()) {
             LOG.debug("==> isAccessAllowed(" + request + ")");
@@ -317,6 +329,38 @@ public class RangerAtlasAuthorizer implements AtlasAuthorizer {
         }
     }
 
+    @Override
+    public void filterTypesDef(AtlasTypesDefFilterRequest request) throws AtlasAuthorizationException {
+
+        AtlasTypesDef typesDef = request.getTypesDef();
+
+        filterTypes(request, typesDef.getEnumDefs());
+        filterTypes(request, typesDef.getStructDefs());
+        filterTypes(request, typesDef.getEntityDefs());
+        filterTypes(request, typesDef.getClassificationDefs());
+        filterTypes(request, typesDef.getRelationshipDefs());
+        filterTypes(request, typesDef.getBusinessMetadataDefs());
+
+    }
+
+    private void filterTypes(AtlasAccessRequest request, List<? extends AtlasBaseTypeDef> typeDefs)throws AtlasAuthorizationException {
+        if (typeDefs != null) {
+            for (ListIterator<? extends AtlasBaseTypeDef> iter = typeDefs.listIterator(); iter.hasNext();) {
+                AtlasBaseTypeDef       typeDef     = iter.next();
+                AtlasTypeAccessRequest typeRequest = new AtlasTypeAccessRequest(request.getAction(), typeDef, request.getUser(), request.getUserGroups());
+
+                typeRequest.setClientIPAddress(request.getClientIPAddress());
+                typeRequest.setForwardedAddresses(request.getForwardedAddresses());
+                typeRequest.setRemoteIPAddress(request.getRemoteIPAddress());
+
+                if (!isAccessAllowed(typeRequest)) {
+                    iter.remove();
+                }
+            }
+        }
+    }
+
+
     private RangerServiceDef getServiceDef() {
         RangerBasePlugin plugin = atlasPlugin;
 
diff --git a/plugin-atlas/src/main/java/org/apache/ranger/services/atlas/RangerServiceAtlas.java b/plugin-atlas/src/main/java/org/apache/ranger/services/atlas/RangerServiceAtlas.java
index 7c89ffe..5b58c06 100644
--- a/plugin-atlas/src/main/java/org/apache/ranger/services/atlas/RangerServiceAtlas.java
+++ b/plugin-atlas/src/main/java/org/apache/ranger/services/atlas/RangerServiceAtlas.java
@@ -76,6 +76,7 @@ public class RangerServiceAtlas extends RangerBaseService {
 	public static final String SEARCH_FEATURE_POLICY_NAME             = "Allow users to manage favorite searches";
 
 	public static final String ACCESS_TYPE_ENTITY_READ  = "entity-read";
+	public static final String ACCESS_TYPE_TYPE_READ = "type-read";
 	public static final String ACCESS_TYPE_ENTITY_CREATE  = "entity-create";
 	public static final String ACCESS_TYPE_ENTITY_UPDATE = "entity-update";
 	public static final String ACCESS_TYPE_ENTITY_DELETE = "entity-delete";
@@ -198,6 +199,14 @@ public class RangerServiceAtlas extends RangerBaseService {
 				policyItemForLookupUser.setDelegateAdmin(false);
 				defaultPolicy.getPolicyItems().add(policyItemForLookupUser);
 			}
+
+			//  add a policy-item for rangertagsync user with 'type-read' permission in the policy for 'type-category'
+			if (policyResources.containsKey(RangerServiceAtlas.RESOURCE_TYPE_CATEGORY)) {
+				RangerPolicyItem policyItemTypeReadForAll = new RangerPolicyItem();
+				policyItemTypeReadForAll.setGroups(Collections.singletonList(RangerPolicyEngine.GROUP_PUBLIC));
+				policyItemTypeReadForAll.setAccesses(Collections.singletonList(new RangerPolicyItemAccess(ACCESS_TYPE_TYPE_READ)));
+				defaultPolicy.getPolicyItems().add(policyItemTypeReadForAll);
+			}
         }
 
         //4.add new policy for public group with entity-read, entity-create, entity-update, entity-delete for  __AtlasUserProfile, __AtlasUserSavedSearch entity type
diff --git a/pom.xml b/pom.xml
index 8936bc4..34fd606 100644
--- a/pom.xml
+++ b/pom.xml
@@ -80,7 +80,7 @@
         <asm.all.version>3.2</asm.all.version>
         <aspectj.version>1.8.2</aspectj.version>
         <assembly.plugin.version>2.6</assembly.plugin.version>
-        <atlas.version>2.1.0</atlas.version>
+        <atlas.version>3.0.0-SNAPSHOT</atlas.version>
         <atlas.gson.version>2.5</atlas.gson.version>
         <atlas.jackson.version>2.10.3</atlas.jackson.version>
         <atlas.jackson.databind.version>2.10.3</atlas.jackson.databind.version>
diff --git a/ranger-atlas-plugin-shim/src/main/java/org/apache/ranger/authorization/atlas/authorizer/RangerAtlasAuthorizer.java b/ranger-atlas-plugin-shim/src/main/java/org/apache/ranger/authorization/atlas/authorizer/RangerAtlasAuthorizer.java
index 0e220f1..916c856 100644
--- a/ranger-atlas-plugin-shim/src/main/java/org/apache/ranger/authorization/atlas/authorizer/RangerAtlasAuthorizer.java
+++ b/ranger-atlas-plugin-shim/src/main/java/org/apache/ranger/authorization/atlas/authorizer/RangerAtlasAuthorizer.java
@@ -25,6 +25,7 @@ import org.apache.atlas.authorize.AtlasSearchResultScrubRequest;
 import org.apache.atlas.authorize.AtlasRelationshipAccessRequest;
 import org.apache.atlas.authorize.AtlasTypeAccessRequest;
 import org.apache.atlas.authorize.AtlasAuthorizationException;
+import org.apache.atlas.authorize.AtlasTypesDefFilterRequest;
 import org.apache.atlas.authorize.AtlasAuthorizer;
 import org.apache.ranger.plugin.classloader.RangerPluginClassLoader;
 import org.slf4j.Logger;
@@ -220,6 +221,27 @@ public class RangerAtlasAuthorizer implements AtlasAuthorizer {
 		}
 	}
 
+	@Override
+	public void filterTypesDef(AtlasTypesDefFilterRequest request) throws AtlasAuthorizationException {
+
+		if (isDebugEnabled) {
+			LOG.debug("==> filterTypesDef(" + request + ")");
+		}
+
+		try {
+			activatePluginClassLoader();
+
+			rangerAtlasAuthorizerImpl.filterTypesDef(request);
+		} finally {
+			deactivatePluginClassLoader();
+		}
+
+		if (isDebugEnabled) {
+			LOG.debug("<== filterTypesDef(): " + request);
+		}
+
+	}
+
     private void activatePluginClassLoader() {
 		if(rangerPluginClassLoader != null) {
 			rangerPluginClassLoader.activate();
@@ -231,4 +253,7 @@ public class RangerAtlasAuthorizer implements AtlasAuthorizer {
 			rangerPluginClassLoader.deactivate();
 		}
 	}
+
+
+
 }
diff --git a/security-admin/db/mysql/optimized/current/ranger_core_db_mysql.sql b/security-admin/db/mysql/optimized/current/ranger_core_db_mysql.sql
index dfaf3c9..3320fc2 100644
--- a/security-admin/db/mysql/optimized/current/ranger_core_db_mysql.sql
+++ b/security-admin/db/mysql/optimized/current/ranger_core_db_mysql.sql
@@ -1758,4 +1758,5 @@ INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active
 INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('J10036',UTC_TIMESTAMP(),'Ranger 1.0.0',UTC_TIMESTAMP(),'localhost','Y');
 INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('J10037',UTC_TIMESTAMP(),'Ranger 1.0.0',UTC_TIMESTAMP(),'localhost','Y');
 INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('J10038',UTC_TIMESTAMP(),'Ranger 1.0.0',UTC_TIMESTAMP(),'localhost','Y');
+INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('J10040',UTC_TIMESTAMP(),'Ranger 1.0.0',UTC_TIMESTAMP(),'localhost','Y');
 INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('JAVA_PATCHES',UTC_TIMESTAMP(),'Ranger 1.0.0',UTC_TIMESTAMP(),'localhost','Y');
diff --git a/security-admin/db/oracle/optimized/current/ranger_core_db_oracle.sql b/security-admin/db/oracle/optimized/current/ranger_core_db_oracle.sql
index 21626f6..698a687 100644
--- a/security-admin/db/oracle/optimized/current/ranger_core_db_oracle.sql
+++ b/security-admin/db/oracle/optimized/current/ranger_core_db_oracle.sql
@@ -1933,5 +1933,6 @@ INSERT INTO x_db_version_h (id,version,inst_at,inst_by,updated_at,updated_by,act
 INSERT INTO x_db_version_h (id,version,inst_at,inst_by,updated_at,updated_by,active) VALUES (X_DB_VERSION_H_SEQ.nextval,'J10036',sys_extract_utc(systimestamp),'Ranger 1.0.0',sys_extract_utc(systimestamp),'localhost','Y');
 INSERT INTO x_db_version_h (id,version,inst_at,inst_by,updated_at,updated_by,active) VALUES (X_DB_VERSION_H_SEQ.nextval,'J10037',sys_extract_utc(systimestamp),'Ranger 1.0.0',sys_extract_utc(systimestamp),'localhost','Y');
 INSERT INTO x_db_version_h (id,version,inst_at,inst_by,updated_at,updated_by,active) VALUES (X_DB_VERSION_H_SEQ.nextval,'J10038',sys_extract_utc(systimestamp),'Ranger 1.0.0',sys_extract_utc(systimestamp),'localhost','Y');
+INSERT INTO x_db_version_h (id,version,inst_at,inst_by,updated_at,updated_by,active) VALUES (X_DB_VERSION_H_SEQ.nextval,'J10040',sys_extract_utc(systimestamp),'Ranger 1.0.0',sys_extract_utc(systimestamp),'localhost','Y');
 INSERT INTO x_db_version_h (id,version,inst_at,inst_by,updated_at,updated_by,active) VALUES (X_DB_VERSION_H_SEQ.nextval,'JAVA_PATCHES',sys_extract_utc(systimestamp),'Ranger 1.0.0',sys_extract_utc(systimestamp),'localhost','Y');
 commit;
diff --git a/security-admin/db/postgres/optimized/current/ranger_core_db_postgres.sql b/security-admin/db/postgres/optimized/current/ranger_core_db_postgres.sql
index 5cd2cc7..edccc0a 100644
--- a/security-admin/db/postgres/optimized/current/ranger_core_db_postgres.sql
+++ b/security-admin/db/postgres/optimized/current/ranger_core_db_postgres.sql
@@ -1881,6 +1881,7 @@ INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active
 INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('J10036',current_timestamp,'Ranger 1.0.0',current_timestamp,'localhost','Y');
 INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('J10037',current_timestamp,'Ranger 1.0.0',current_timestamp,'localhost','Y');
 INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('J10038',current_timestamp,'Ranger 1.0.0',current_timestamp,'localhost','Y');
+INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('J10040',current_timestamp,'Ranger 1.0.0',current_timestamp,'localhost','Y');
 INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('JAVA_PATCHES',current_timestamp,'Ranger 1.0.0',current_timestamp,'localhost','Y');
 
 DROP VIEW IF EXISTS vx_trx_log;
diff --git a/security-admin/db/sqlanywhere/optimized/current/ranger_core_db_sqlanywhere.sql b/security-admin/db/sqlanywhere/optimized/current/ranger_core_db_sqlanywhere.sql
index 081b153..1ffbc6c 100644
--- a/security-admin/db/sqlanywhere/optimized/current/ranger_core_db_sqlanywhere.sql
+++ b/security-admin/db/sqlanywhere/optimized/current/ranger_core_db_sqlanywhere.sql
@@ -2279,6 +2279,8 @@ INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active
 GO
 INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('J10038',CURRENT_TIMESTAMP,'Ranger 1.0.0',CURRENT_TIMESTAMP,'localhost','Y');
 GO
+INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('J10040',CURRENT_TIMESTAMP,'Ranger 1.0.0',CURRENT_TIMESTAMP,'localhost','Y');
+GO
 INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('JAVA_PATCHES',CURRENT_TIMESTAMP,'Ranger 1.0.0',CURRENT_TIMESTAMP,'localhost','Y');
 GO
 exit
diff --git a/security-admin/db/sqlserver/optimized/current/ranger_core_db_sqlserver.sql b/security-admin/db/sqlserver/optimized/current/ranger_core_db_sqlserver.sql
index 642d6c1..957f924 100644
--- a/security-admin/db/sqlserver/optimized/current/ranger_core_db_sqlserver.sql
+++ b/security-admin/db/sqlserver/optimized/current/ranger_core_db_sqlserver.sql
@@ -3964,6 +3964,7 @@ INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active
 INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('J10036',CURRENT_TIMESTAMP,'Ranger 1.0.0',CURRENT_TIMESTAMP,'localhost','Y');
 INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('J10037',CURRENT_TIMESTAMP,'Ranger 1.0.0',CURRENT_TIMESTAMP,'localhost','Y');
 INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('J10038',CURRENT_TIMESTAMP,'Ranger 1.0.0',CURRENT_TIMESTAMP,'localhost','Y');
+INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('J10040',CURRENT_TIMESTAMP,'Ranger 1.0.0',CURRENT_TIMESTAMP,'localhost','Y');
 INSERT INTO x_db_version_h (version,inst_at,inst_by,updated_at,updated_by,active) VALUES ('JAVA_PATCHES',CURRENT_TIMESTAMP,'Ranger 1.0.0',CURRENT_TIMESTAMP,'localhost','Y');
 GO
 CREATE VIEW [dbo].[vx_trx_log] AS
diff --git a/security-admin/src/main/java/org/apache/ranger/patch/PatchForAtlasToAddTypeRead_J10040.java b/security-admin/src/main/java/org/apache/ranger/patch/PatchForAtlasToAddTypeRead_J10040.java
new file mode 100644
index 0000000..f303092
--- /dev/null
+++ b/security-admin/src/main/java/org/apache/ranger/patch/PatchForAtlasToAddTypeRead_J10040.java
@@ -0,0 +1,241 @@
+/*
+ * 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.
+ */
+
+package org.apache.ranger.patch;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.log4j.Logger;
+import org.apache.ranger.biz.ServiceDBStore;
+import org.apache.ranger.common.RangerValidatorFactory;
+import org.apache.ranger.db.RangerDaoManager;
+import org.apache.ranger.entity.XXServiceDef;
+import org.apache.ranger.plugin.model.RangerServiceDef;
+import org.apache.ranger.plugin.model.RangerServiceDef.RangerAccessTypeDef;
+import org.apache.ranger.plugin.model.validation.RangerServiceDefValidator;
+import org.apache.ranger.plugin.model.validation.RangerValidator.Action;
+import org.apache.ranger.plugin.store.EmbeddedServiceDefsUtil;
+import org.apache.ranger.util.CLIUtil;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+import org.apache.ranger.entity.XXPolicy;
+import org.apache.ranger.entity.XXService;
+import org.apache.ranger.plugin.model.RangerPolicy;
+import org.apache.ranger.plugin.model.RangerPolicy.RangerPolicyItem;
+import org.apache.ranger.plugin.model.RangerPolicy.RangerPolicyItemAccess;
+
+@Component
+public class PatchForAtlasToAddTypeRead_J10040 extends org.apache.ranger.patch.BaseLoader {
+    private static final Logger logger = Logger.getLogger(PatchForAtlasToAddTypeRead_J10040.class);
+
+    private static final List<String> ATLAS_RESOURCES = new ArrayList<>(
+            Arrays.asList("type"));
+    private static final List<String> ATLAS_ACCESS_TYPES = new ArrayList<>(
+            Arrays.asList("type-read"));
+
+    private static final String GROUP_PUBLIC = "public";
+    private static final String TYPE_READ = "type-read";
+    private static final String ALL_TYPE_RESOURCE_DEF_NAME = "all - type-category, type";
+
+
+    @Autowired
+    RangerDaoManager daoMgr;
+
+    @Autowired
+    ServiceDBStore svcDBStore;
+
+    @Autowired
+    RangerValidatorFactory validatorFactory;
+
+    @Autowired
+    ServiceDBStore svcStore;
+
+    public static void main(String[] args) {
+        logger.info("main()");
+        try {
+            PatchForAtlasToAddTypeRead_J10040 loader = (PatchForAtlasToAddTypeRead_J10040) CLIUtil
+                    .getBean(PatchForAtlasToAddTypeRead_J10040.class);
+            loader.init();
+            while (loader.isMoreToProcess()) {
+                loader.load();
+            }
+            logger.info("Load complete. Exiting!!!");
+            System.exit(0);
+        } catch (Exception e) {
+            logger.error("Error loading", e);
+            System.exit(1);
+        }
+    }
+
+    @Override
+    public void init() throws Exception {
+        // Do Nothing
+    }
+
+    @Override
+    public void execLoad() {
+        logger.info("==> PatchForAtlasToAddTypeRead_J10040.execLoad()");
+        try {
+            addTypeReadPermissionInServiceDef();
+            updateDefaultPolicyForType();
+        } catch (Exception e) {
+            throw new RuntimeException(
+                    "Error while updating " + EmbeddedServiceDefsUtil.EMBEDDED_SERVICEDEF_ATLAS_NAME + " service-def", e);
+        }
+        logger.info("<== PatchForAtlasToAddTypeRead_J10040.execLoad()");
+    }
+
+    @Override
+    public void printStats() {
+        logger.info("PatchForAtlasToAddTypeRead_J10040 Logs");
+    }
+
+    private void addTypeReadPermissionInServiceDef() throws Exception {
+
+        logger.debug("==>> addTypeReadPermissionInServiceDef");
+        RangerServiceDef ret = null;
+        RangerServiceDef embeddedAtlasServiceDef = null;
+        XXServiceDef xXServiceDefObj = null;
+        RangerServiceDef dbAtlasServiceDef = null;
+        List<RangerServiceDef.RangerResourceDef> embeddedAtlasResourceDefs = null;
+        List<RangerServiceDef.RangerAccessTypeDef> embeddedAtlasAccessTypes = null;
+
+        embeddedAtlasServiceDef = EmbeddedServiceDefsUtil.instance()
+                .getEmbeddedServiceDef(EmbeddedServiceDefsUtil.EMBEDDED_SERVICEDEF_ATLAS_NAME);
+        if (embeddedAtlasServiceDef != null) {
+            xXServiceDefObj = daoMgr.getXXServiceDef()
+                    .findByName(EmbeddedServiceDefsUtil.EMBEDDED_SERVICEDEF_ATLAS_NAME);
+            if (xXServiceDefObj == null) {
+                logger.info(xXServiceDefObj + ": service-def not found. No patching is needed");
+                return;
+            }
+
+            dbAtlasServiceDef = svcDBStore.getServiceDefByName(EmbeddedServiceDefsUtil.EMBEDDED_SERVICEDEF_ATLAS_NAME);
+
+            embeddedAtlasResourceDefs = embeddedAtlasServiceDef.getResources();
+            embeddedAtlasAccessTypes = embeddedAtlasServiceDef.getAccessTypes();
+
+            if (checkResourcePresent(embeddedAtlasResourceDefs)) {
+                dbAtlasServiceDef.setResources(embeddedAtlasResourceDefs);
+                if (checkAccessPresent(embeddedAtlasAccessTypes)) {
+                    dbAtlasServiceDef.setAccessTypes(embeddedAtlasAccessTypes);
+                }
+            }
+
+            RangerServiceDefValidator validator = validatorFactory.getServiceDefValidator(svcStore);
+            validator.validate(dbAtlasServiceDef, Action.UPDATE);
+            ret = svcStore.updateServiceDef(dbAtlasServiceDef);
+            if (ret == null) {
+                logger.error("Error while updating " + EmbeddedServiceDefsUtil.EMBEDDED_SERVICEDEF_ATLAS_NAME
+                        + " service-def");
+                throw new RuntimeException("Error while updating "
+                        + EmbeddedServiceDefsUtil.EMBEDDED_SERVICEDEF_ATLAS_NAME + " service-def");
+            }
+        }
+        logger.debug("<<== addTypeReadPermissionInServiceDef");
+    }
+
+    private boolean checkResourcePresent(List<RangerServiceDef.RangerResourceDef> resourceDefs) {
+        boolean ret = false;
+        for (RangerServiceDef.RangerResourceDef resourceDef : resourceDefs) {
+            if (ATLAS_RESOURCES.contains(resourceDef.getName())) {
+                ret = true;
+                break;
+            }
+        }
+        return ret;
+    }
+
+    private boolean checkAccessPresent(List<RangerAccessTypeDef> embeddedAtlasAccessTypes) {
+        boolean ret = false;
+        for (RangerServiceDef.RangerAccessTypeDef accessDef : embeddedAtlasAccessTypes) {
+            if (ATLAS_ACCESS_TYPES.contains(accessDef.getName())) {
+                ret = true;
+                break;
+            }
+        }
+        return ret;
+    }
+
+    private void updateDefaultPolicyForType() throws Exception {
+        logger.info("==> updateDefaultPolicyForType() ");
+
+        XXServiceDef xXServiceDefObj = daoMgr.getXXServiceDef()
+                .findByName(EmbeddedServiceDefsUtil.EMBEDDED_SERVICEDEF_ATLAS_NAME);
+        if (xXServiceDefObj == null) {
+            logger.debug("ServiceDef not found with name :" + EmbeddedServiceDefsUtil.EMBEDDED_SERVICEDEF_ATLAS_NAME);
+            return;
+        }
+        Long xServiceDefId = xXServiceDefObj.getId();
+        List<XXService> xxServices = daoMgr.getXXService().findByServiceDefId(xServiceDefId);
+
+        for (XXService xxService : xxServices) {
+            List<XXPolicy> xxPolicies = daoMgr.getXXPolicy().findByServiceId(xxService.getId());
+
+            for (XXPolicy xxPolicy : xxPolicies) {
+                if (xxPolicy.getName().equalsIgnoreCase(ALL_TYPE_RESOURCE_DEF_NAME)) {
+
+                    RangerPolicy rPolicy = svcDBStore.getPolicy(xxPolicy.getId());
+                    List<RangerPolicyItem> policyItems = rPolicy.getPolicyItems();
+
+                    for (RangerPolicyItem item : policyItems) {
+                        if (!checkIfTypeReadPermissionSet(item)) {
+                            List<RangerPolicyItemAccess> itemAccesses = item.getAccesses();
+                            itemAccesses.add(getTypeReadPolicyItemAccesses());
+                            item.setAccesses(itemAccesses);
+                        }
+                    }
+
+                    RangerPolicyItem rangerPolicyItemReadType = new RangerPolicyItem();
+                    rangerPolicyItemReadType.setDelegateAdmin(Boolean.FALSE);
+                    rangerPolicyItemReadType.setAccesses(Arrays.asList(getTypeReadPolicyItemAccesses()));
+                    rangerPolicyItemReadType.setGroups(Arrays.asList(GROUP_PUBLIC));
+
+                    policyItems.add(rangerPolicyItemReadType);
+
+                    svcDBStore.updatePolicy(rPolicy);
+                }
+
+            }
+
+        }
+        logger.info("<== updateDefaultPolicyForType() ");
+    }
+
+    private RangerPolicyItemAccess getTypeReadPolicyItemAccesses() {
+
+        RangerPolicyItemAccess policyItemAccess = new RangerPolicyItemAccess();
+        policyItemAccess.setType(TYPE_READ);
+        policyItemAccess.setIsAllowed(true);
+
+        return policyItemAccess;
+    }
+
+    boolean checkIfTypeReadPermissionSet(RangerPolicyItem item) {
+        boolean ret = false;
+        for (RangerPolicyItemAccess itemAccess : item.getAccesses()) {
+            if (ATLAS_ACCESS_TYPES.contains(itemAccess.getType())) {
+                ret = true;
+                break;
+            }
+        }
+        return ret;
+    }
+
+}
\ No newline at end of file