You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by pr...@apache.org on 2014/03/27 01:22:30 UTC

git commit: SENTRY-145: Store needs to handle privilege normalization (Brock Noland via Prasad Mujumdar)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master 4247d21fa -> 04c4041bf


SENTRY-145: Store needs to handle privilege normalization (Brock Noland via Prasad Mujumdar)


Project: http://git-wip-us.apache.org/repos/asf/incubator-sentry/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-sentry/commit/04c4041b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-sentry/tree/04c4041b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-sentry/diff/04c4041b

Branch: refs/heads/master
Commit: 04c4041bf470a8657ce25b8e8759ee1823e25141
Parents: 4247d21
Author: Prasad Mujumdar <pr...@cloudera.com>
Authored: Wed Mar 26 17:21:58 2014 -0700
Committer: Prasad Mujumdar <pr...@cloudera.com>
Committed: Wed Mar 26 17:21:58 2014 -0700

----------------------------------------------------------------------
 .../db/service/persistent/SentryStore.java      | 138 ++++++++++++++++++-
 .../thrift/SentryPolicyStoreProcessor.java      |  80 +----------
 .../db/service/persistent/TestSentryStore.java  |  31 +++--
 3 files changed, 154 insertions(+), 95 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/04c4041b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
index 9c678d5..aca1350 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/SentryStore.java
@@ -37,9 +37,11 @@ import javax.jdo.Transaction;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
 import org.apache.sentry.provider.common.ProviderConstants;
 import org.apache.sentry.provider.db.SentryAlreadyExistsException;
+import org.apache.sentry.provider.db.SentryInvalidInputException;
 import org.apache.sentry.provider.db.SentryNoSuchObjectException;
 import org.apache.sentry.provider.db.service.model.MSentryGroup;
 import org.apache.sentry.provider.db.service.model.MSentryPrivilege;
@@ -49,6 +51,7 @@ import org.apache.sentry.provider.db.service.thrift.TSentryGroup;
 import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
 import org.apache.sentry.provider.db.service.thrift.TSentryRole;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
+import org.apache.sentry.service.thrift.ServiceConstants.PrivilegeScope;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -199,7 +202,7 @@ public class SentryStore {
 
   //TODO: handle case where a) privilege already exists, b) role to privilege mapping already exists
   public CommitContext alterSentryRoleGrantPrivilege(String roleName,
-      TSentryPrivilege privilege) throws SentryNoSuchObjectException {
+      TSentryPrivilege privilege) throws SentryNoSuchObjectException, SentryInvalidInputException {
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
     roleName = roleName.trim().toLowerCase();
@@ -232,7 +235,7 @@ public class SentryStore {
   }
 
   public CommitContext alterSentryRoleRevokePrivilege(String roleName,
-      String privilegeName) throws SentryNoSuchObjectException {
+      TSentryPrivilege tPrivilege) throws SentryNoSuchObjectException, SentryInvalidInputException {
     boolean rollbackTransaction = true;
     PersistenceManager pm = null;
     try {
@@ -249,16 +252,21 @@ public class SentryStore {
         query.setFilter("this.privilegeName == t");
         query.declareParameters("java.lang.String t");
         query.setUnique(true);
+        String privilegeName = constructPrivilegeName(tPrivilege);
         MSentryPrivilege mPrivilege = (MSentryPrivilege) query.execute(privilegeName);
         if (mPrivilege == null) {
-          throw new SentryNoSuchObjectException("Privilege: " + privilegeName);
+          revokePartialPrivilege(pm, mRole, tPrivilege);
+          CommitContext commit = commitUpdateTransaction(pm);
+          rollbackTransaction = false;
+          return commit;
         } else {
-          // remove privilege and role objects from each other's set. needed by datanucleus to model
-          // m:n relationships correctly through a join table.
+          // remove privilege and role objects from each other's set. needed by
+          // datanucleus to model m:n relationships correctly through a join table.
           mRole.removePrivilege(mPrivilege);
           CommitContext commit = commitUpdateTransaction(pm);
           rollbackTransaction = false;
           return commit;
+
         }
       }
     } finally {
@@ -268,6 +276,120 @@ public class SentryStore {
     }
   }
 
+  /**
+   * Our privilege model at present only allows ALL on server and databases.
+   * However, roles can be granted ALL, SELECT, and INSERT on tables. When
+   * a role has ALL and SELECT or INSERT are revoked, we need to remove the ALL
+   * privilege and add SELECT (INSERT was revoked) or INSERT (SELECT was revoked).
+   */
+  private void revokePartialPrivilege(PersistenceManager pm, MSentryRole role,
+      TSentryPrivilege tPrivilege)
+      throws SentryNoSuchObjectException, SentryInvalidInputException {
+    // only perform partial revoke if INSERT/SELECT were the action
+    // and the privilege being revoked is on a table
+    String action = tPrivilege.getAction();
+    if (AccessConstants.ALL.equalsIgnoreCase(action) ||
+        StringUtils.isEmpty(tPrivilege.getDbName()) || StringUtils.isEmpty(tPrivilege.getTableName())) {
+      throw new SentryNoSuchObjectException("Unknown privilege: " + tPrivilege);
+    }
+    TSentryPrivilege tPrivilegeAll = new TSentryPrivilege(tPrivilege);
+    tPrivilegeAll.setAction(AccessConstants.ALL);
+    String allPrivilegeName = constructPrivilegeName(tPrivilegeAll);
+    Query query = pm.newQuery(MSentryPrivilege.class);
+    query.setFilter("this.privilegeName == t");
+    query.declareParameters("java.lang.String t");
+    query.setUnique(true);
+    MSentryPrivilege allPrivilege = (MSentryPrivilege) query.execute(allPrivilegeName);
+    if (allPrivilege == null) {
+      throw new SentryNoSuchObjectException("Unknown privilege: " + tPrivilege);
+    }
+    role.removePrivilege(allPrivilege);
+    if (AccessConstants.SELECT.equalsIgnoreCase(action)) {
+      tPrivilege.setAction(AccessConstants.INSERT);
+    } else if (AccessConstants.INSERT.equalsIgnoreCase(action)) {
+      tPrivilege.setAction(AccessConstants.SELECT);
+    } else {
+      throw new IllegalStateException("Unexpected action: " + action);
+    }
+    role.appendPrivilege(convertToMSentryPrivilege(tPrivilege));
+  }
+
+  //TODO:Validate privilege scope?
+  @VisibleForTesting
+  public static String constructPrivilegeName(TSentryPrivilege privilege) throws SentryInvalidInputException {
+    StringBuilder privilegeName = new StringBuilder();
+    String serverName = privilege.getServerName();
+    String dbName = privilege.getDbName();
+    String tableName = privilege.getTableName();
+    String uri = privilege.getURI();
+    String action = privilege.getAction();
+    PrivilegeScope scope;
+
+    if (serverName == null) {
+      throw new SentryInvalidInputException("Server name is null");
+    }
+
+    if (AccessConstants.SELECT.equalsIgnoreCase(action) ||
+        AccessConstants.INSERT.equalsIgnoreCase(action)) {
+      if (Strings.nullToEmpty(tableName).trim().isEmpty()) {
+        throw new SentryInvalidInputException("Table name can't be null for SELECT/INSERT privilege");
+      }
+    }
+
+    // Validate privilege scope
+    try {
+      scope = Enum.valueOf(PrivilegeScope.class, privilege.getPrivilegeScope());
+    } catch (IllegalArgumentException e) {
+      throw new SentryInvalidInputException("Invalid Privilege scope: " +
+          privilege.getPrivilegeScope());
+    }
+    if (PrivilegeScope.SERVER.equals(scope)) {
+      if (StringUtils.isNotEmpty(dbName) || StringUtils.isNotEmpty(tableName)) {
+        throw new SentryInvalidInputException("DB and TABLE names should not be "
+            + "set for SERVER scope");
+      }
+    } else if (PrivilegeScope.DATABASE.equals(scope)) {
+      if (StringUtils.isEmpty(dbName)) {
+        throw new SentryInvalidInputException("DB name not set for DB scope");
+      }
+      if (StringUtils.isNotEmpty(tableName)) {
+        StringUtils.isNotEmpty("TABLE names should not be set for DB scope");
+      }
+    } else if (PrivilegeScope.TABLE.equals(scope)) {
+      if (StringUtils.isEmpty(dbName) || StringUtils.isEmpty(tableName)) {
+        throw new SentryInvalidInputException("TABLE or DB name not set for TABLE scope");
+      }
+    } else if (PrivilegeScope.URI.equals(scope)){
+      if (StringUtils.isEmpty(uri)) {
+        throw new SentryInvalidInputException("URI path not set for URI scope");
+      }
+      if (StringUtils.isNotEmpty(tableName)) {
+        throw new SentryInvalidInputException("TABLE should not be set for URI scope");
+      }
+    } else {
+      throw new SentryInvalidInputException("Unsupported operation scope: " + scope);
+    }
+
+    if (uri == null || uri.equals("")) {
+      privilegeName.append(serverName);
+      privilegeName.append("+");
+      privilegeName.append(dbName);
+
+      if (tableName != null && !tableName.equals("")) {
+        privilegeName.append("+");
+        privilegeName.append(tableName);
+      }
+      privilegeName.append("+");
+      privilegeName.append(action);
+    } else {
+      privilegeName.append(serverName);
+      privilegeName.append("+");
+      privilegeName.append(uri);
+    }
+    return privilegeName.toString();
+  }
+
+
   public CommitContext dropSentryRole(String roleName)
   throws SentryNoSuchObjectException {
     boolean rollbackTransaction = true;
@@ -534,8 +656,10 @@ public class SentryStore {
   /**
    * Converts thrift object to model object. Additionally does normalization
    * such as trimming whitespace and setting appropriate case.
+   * @throws SentryInvalidInputException
    */
-  private MSentryPrivilege convertToMSentryPrivilege(TSentryPrivilege privilege) {
+  private MSentryPrivilege convertToMSentryPrivilege(TSentryPrivilege privilege)
+      throws SentryInvalidInputException {
     MSentryPrivilege mSentryPrivilege = new MSentryPrivilege();
     mSentryPrivilege.setServerName(safeTrim(privilege.getServerName()));
     mSentryPrivilege.setDbName(safeTrim(privilege.getDbName()));
@@ -545,7 +669,7 @@ public class SentryStore {
     mSentryPrivilege.setCreateTime(privilege.getCreateTime());
     mSentryPrivilege.setGrantorPrincipal(safeTrim(privilege.getGrantorPrincipal()));
     mSentryPrivilege.setURI(safeTrim(privilege.getURI()));
-    mSentryPrivilege.setPrivilegeName(safeTrim(privilege.getPrivilegeName()));
+    mSentryPrivilege.setPrivilegeName(constructPrivilegeName(privilege));
     return mSentryPrivilege;
   }
   private String safeTrim(String s) {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/04c4041b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
index 9793ab4..6c52fa4 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
@@ -32,7 +32,6 @@ import org.apache.sentry.provider.db.SentryNoSuchObjectException;
 import org.apache.sentry.provider.db.service.persistent.CommitContext;
 import org.apache.sentry.provider.db.service.persistent.SentryStore;
 import org.apache.sentry.provider.db.service.thrift.PolicyStoreConstants.PolicyStoreServerConfig;
-import org.apache.sentry.service.thrift.ServiceConstants.PrivilegeScope;
 import org.apache.sentry.service.thrift.Status;
 import org.apache.sentry.service.thrift.TSentryResponseStatus;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
@@ -108,79 +107,6 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     return handlers;
   }
 
-  @VisibleForTesting
-  public static String constructPrivilegeName(TSentryPrivilege privilege) throws SentryInvalidInputException {
-    StringBuilder privilegeName = new StringBuilder();
-    String serverName = privilege.getServerName();
-    String dbName = privilege.getDbName();
-    String tableName = privilege.getTableName();
-    String uri = privilege.getURI();
-    String action = privilege.getAction();
-    PrivilegeScope scope;
-
-    if (serverName == null) {
-      throw new SentryInvalidInputException("Server name is null");
-    }
-
-    if ("SELECT".equalsIgnoreCase(action) || "INSERT".equalsIgnoreCase(action)) {
-      if (tableName == null || tableName.equals("")) {
-        throw new SentryInvalidInputException("Table name can't be null for SELECT/INSERT privilege");
-      }
-    }
-
-    // Validate privilege scope
-    try {
-      scope = Enum.valueOf(PrivilegeScope.class, privilege.getPrivilegeScope());
-    } catch (IllegalArgumentException e) {
-      throw new SentryInvalidInputException("Invalid Privilege scope: " +
-          privilege.getPrivilegeScope());
-    }
-    if (PrivilegeScope.SERVER.equals(scope)) {
-      if (StringUtils.isNotEmpty(dbName) || StringUtils.isNotEmpty(tableName)) {
-        throw new SentryInvalidInputException("DB and TABLE names should not be "
-            + "set for SERVER scope");
-      }
-    } else if (PrivilegeScope.DATABASE.equals(scope)) {
-      if (StringUtils.isEmpty(dbName)) {
-        throw new SentryInvalidInputException("DB name not set for DB scope");
-      }
-      if (StringUtils.isNotEmpty(tableName)) {
-        StringUtils.isNotEmpty("TABLE names should not be set for DB scope");
-      }
-    } else if (PrivilegeScope.TABLE.equals(scope)) {
-      if (StringUtils.isEmpty(dbName) || StringUtils.isEmpty(tableName)) {
-        throw new SentryInvalidInputException("TABLE or DB name not set for TABLE scope");
-      }
-    } else if (PrivilegeScope.URI.equals(scope)){
-      if (StringUtils.isEmpty(uri)) {
-        throw new SentryInvalidInputException("URI path not set for URI scope");
-      }
-      if (StringUtils.isNotEmpty(tableName)) {
-        throw new SentryInvalidInputException("TABLE should not be set for URI scope");
-      }
-    } else {
-      throw new SentryInvalidInputException("Unsupported operation scope: " + scope);
-    }
-
-    if (uri == null || uri.equals("")) {
-      privilegeName.append(serverName);
-      privilegeName.append("+");
-      privilegeName.append(dbName);
-
-      if (tableName != null && !tableName.equals("")) {
-        privilegeName.append("+");
-        privilegeName.append(tableName);
-      }
-      privilegeName.append("+");
-      privilegeName.append(action);
-    } else {
-      privilegeName.append(serverName);
-      privilegeName.append("+");
-      privilegeName.append(uri);
-    }
-    return privilegeName.toString();
-  }
-
   private static Set<String> toTrimedLower(Set<String> s) {
     Set<String> result = Sets.newHashSet();
     for (String v : s) {
@@ -233,8 +159,6 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     TAlterSentryRoleGrantPrivilegeResponse response = new TAlterSentryRoleGrantPrivilegeResponse();
     try {
       authorize(request.getRequestorUserName(), request.getRequestorGroupNames());
-      String privilegeName = constructPrivilegeName(request.getPrivilege());
-      request.getPrivilege().setPrivilegeName(privilegeName);
       CommitContext commitContext = sentryStore.alterSentryRoleGrantPrivilege(request.getRoleName(),
                                     request.getPrivilege());
       response.setStatus(Status.OK());
@@ -266,10 +190,8 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     TAlterSentryRoleRevokePrivilegeResponse response = new TAlterSentryRoleRevokePrivilegeResponse();
     try {
       authorize(request.getRequestorUserName(), request.getRequestorGroupNames());
-      String privilegeName = constructPrivilegeName(request.getPrivilege());
-      request.getPrivilege().setPrivilegeName(privilegeName);
       CommitContext commitContext = sentryStore.alterSentryRoleRevokePrivilege(request.getRoleName(),
-                                    request.getPrivilege().getPrivilegeName());
+                                    request.getPrivilege());
       response.setStatus(Status.OK());
       notificationHandlerInvoker.alter_sentry_role_revoke_privilege(commitContext,
           request, response);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/04c4041b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStore.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStore.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStore.java
index f500c2d..40b9460 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStore.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStore.java
@@ -28,11 +28,11 @@ import java.util.Set;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.provider.db.SentryAlreadyExistsException;
 import org.apache.sentry.provider.db.SentryNoSuchObjectException;
 import org.apache.sentry.provider.db.service.model.MSentryPrivilege;
 import org.apache.sentry.provider.db.service.model.MSentryRole;
-import org.apache.sentry.provider.db.service.thrift.SentryPolicyStoreProcessor;
 import org.apache.sentry.provider.db.service.thrift.TSentryActiveRoleSet;
 import org.apache.sentry.provider.db.service.thrift.TSentryGroup;
 import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
@@ -123,24 +123,37 @@ public class TestSentryStore {
   public void testGrantRevokePrivilege() throws Exception {
     String roleName = "test-privilege";
     String grantor = "g1";
+    String server = "server1";
+    String db = "db1";
+    String table = "tbl1";
     long seqId = sentryStore.createSentryRole(roleName, grantor).getSequenceId();
     TSentryPrivilege privilege = new TSentryPrivilege();
     privilege.setPrivilegeScope("TABLE");
-    privilege.setServerName("server1");
-    privilege.setDbName("db1");
-    privilege.setTableName("tbl1");
-    privilege.setAction("SELECT");
+    privilege.setServerName(server);
+    privilege.setDbName(db);
+    privilege.setTableName(table);
+    privilege.setAction(AccessConstants.ALL);
     privilege.setGrantorPrincipal(grantor);
     privilege.setCreateTime(System.currentTimeMillis());
-    privilege.setPrivilegeName(SentryPolicyStoreProcessor.constructPrivilegeName(privilege));
+    privilege.setPrivilegeName(SentryStore.constructPrivilegeName(privilege));
     assertEquals(seqId + 1, sentryStore.alterSentryRoleGrantPrivilege(roleName, privilege)
         .getSequenceId());
     MSentryRole role = sentryStore.getMSentryRoleByName(roleName);
     Set<MSentryPrivilege> privileges = role.getPrivileges();
     assertEquals(privileges.toString(), 1, privileges.size());
     assertEquals(privilege.getPrivilegeName(), Iterables.get(privileges, 0).getPrivilegeName());
-    assertEquals(seqId + 2, sentryStore.alterSentryRoleRevokePrivilege(roleName, privilege.getPrivilegeName())
+    privilege.setAction(AccessConstants.SELECT);
+    assertEquals(seqId + 2, sentryStore.alterSentryRoleRevokePrivilege(roleName, privilege)
         .getSequenceId());
+    // after having ALL and revoking SELECT, we should have INSERT
+    role = sentryStore.getMSentryRoleByName(roleName);
+    privileges = role.getPrivileges();
+    assertEquals(privileges.toString(), 1, privileges.size());
+    MSentryPrivilege mPrivilege = Iterables.get(privileges, 0);
+    assertEquals(server, mPrivilege.getServerName());
+    assertEquals(db, mPrivilege.getDbName());
+    assertEquals(table, mPrivilege.getTableName());
+    assertEquals(AccessConstants.INSERT, mPrivilege.getAction());
   }
 
   @Test
@@ -158,7 +171,7 @@ public class TestSentryStore {
     privilege1.setAction("SELECT");
     privilege1.setGrantorPrincipal(grantor);
     privilege1.setCreateTime(System.currentTimeMillis());
-    privilege1.setPrivilegeName(SentryPolicyStoreProcessor.constructPrivilegeName(privilege1));
+    privilege1.setPrivilegeName(SentryStore.constructPrivilegeName(privilege1));
     assertEquals(seqId + 2, sentryStore.alterSentryRoleGrantPrivilege(roleName1, privilege1)
         .getSequenceId());
     assertEquals(seqId + 3, sentryStore.alterSentryRoleGrantPrivilege(roleName2, privilege1)
@@ -168,7 +181,7 @@ public class TestSentryStore {
     privilege2.setServerName("server1");
     privilege2.setGrantorPrincipal(grantor);
     privilege2.setCreateTime(System.currentTimeMillis());
-    privilege2.setPrivilegeName(SentryPolicyStoreProcessor.constructPrivilegeName(privilege2));
+    privilege2.setPrivilegeName(SentryStore.constructPrivilegeName(privilege2));
     assertEquals(seqId + 4, sentryStore.alterSentryRoleGrantPrivilege(roleName2, privilege2)
         .getSequenceId());
     Set<TSentryGroup> groups = Sets.newHashSet();