You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by li...@apache.org on 2018/05/24 16:58:57 UTC

[2/2] sentry git commit: SENTRY-2156: Update provider-db backend code to grant privileges to user (Na Li, reviewed by Alexander Kolbasov, Sergio Pena, Kalyan Kumar Kalvagadda, Arjun Mishra)

SENTRY-2156: Update provider-db backend code to grant privileges to user (Na Li, reviewed by Alexander Kolbasov, Sergio Pena, Kalyan Kumar Kalvagadda, Arjun Mishra)


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

Branch: refs/heads/master
Commit: 97f666345c1c068fed56a29512f9dcea77e44c8a
Parents: a06e656
Author: lina.li <li...@cloudera.com>
Authored: Thu May 24 11:57:30 2018 -0500
Committer: lina.li <li...@cloudera.com>
Committed: Thu May 24 11:57:30 2018 -0500

----------------------------------------------------------------------
 .../sentry/service/common/ServiceConstants.java |   5 +
 .../thrift/SentryPolicyStoreProcessor.java      |   5 +-
 .../service/persistent/QueryParamBuilder.java   |  23 +
 .../db/service/persistent/SentryStore.java      | 927 ++++++++++++++++---
 .../db/service/persistent/TestSentryStore.java  | 377 ++++++--
 .../e2e/dbprovider/TestDatabaseProvider.java    |  12 +-
 6 files changed, 1184 insertions(+), 165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/97f66634/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/service/common/ServiceConstants.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/service/common/ServiceConstants.java b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/service/common/ServiceConstants.java
index 71e9585..53daae8 100644
--- a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/service/common/ServiceConstants.java
+++ b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/service/common/ServiceConstants.java
@@ -248,4 +248,9 @@ public class ServiceConstants {
 
   public static final String SENTRY_ZK_JAAS_NAME = "Sentry";
   public static final String CURRENT_INCARNATION_ID_KEY = "current.incarnation.key";
+
+  public enum SentryEntityType {
+    ROLE,
+    USER
+  }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/97f66634/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/api/service/thrift/SentryPolicyStoreProcessor.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/api/service/thrift/SentryPolicyStoreProcessor.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/api/service/thrift/SentryPolicyStoreProcessor.java
index 816cfe1..c23683f 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/api/service/thrift/SentryPolicyStoreProcessor.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/api/service/thrift/SentryPolicyStoreProcessor.java
@@ -55,6 +55,7 @@ import org.apache.sentry.api.service.thrift.validator.GrantPrivilegeRequestValid
 import org.apache.sentry.api.service.thrift.validator.RevokePrivilegeRequestValidator;
 import org.apache.sentry.api.common.SentryServiceUtil;
 import org.apache.sentry.service.common.ServiceConstants.ConfUtilties;
+import org.apache.sentry.service.common.ServiceConstants.SentryEntityType;
 import org.apache.sentry.service.common.ServiceConstants.ServerConfig;
 import org.apache.sentry.api.common.Status;
 import org.apache.sentry.service.thrift.TSentryResponseStatus;
@@ -802,7 +803,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       }
       if (request.isSetAuthorizableHierarchy()) {
         TSentryAuthorizable authorizableHierarchy = request.getAuthorizableHierarchy();
-        privilegeSet = sentryStore.getTSentryPrivileges(Sets.newHashSet(request.getRoleName()), authorizableHierarchy);
+        privilegeSet = sentryStore.getTSentryPrivileges(SentryEntityType.ROLE, Sets.newHashSet(request.getRoleName()), authorizableHierarchy);
       } else {
         privilegeSet = sentryStore.getAllTSentryPrivilegesByRoleName(request.getRoleName());
       }
@@ -1049,6 +1050,8 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
         authRoleMap.put(authorizable, sentryStore
             .listSentryPrivilegesByAuthorizable(requestedGroups,
                 request.getRoleSet(), authorizable, inAdminGroups(memberGroups)));
+
+        // TODO: add privileges associated with user by calling listSentryPrivilegesByAuthorizableForUser
       }
       response.setPrivilegesMapByAuth(authRoleMap);
       response.setStatus(Status.OK());

http://git-wip-us.apache.org/repos/asf/sentry/blob/97f66634/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/QueryParamBuilder.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/QueryParamBuilder.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/QueryParamBuilder.java
index 8a77fc1..6075e3f 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/QueryParamBuilder.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/QueryParamBuilder.java
@@ -20,6 +20,7 @@ package org.apache.sentry.provider.db.service.persistent;
 
 import com.google.common.base.Joiner;
 import org.apache.sentry.provider.db.service.model.MSentryRole;
+import org.apache.sentry.provider.db.service.model.MSentryUser;
 
 import javax.annotation.concurrent.NotThreadSafe;
 import javax.jdo.Query;
@@ -320,6 +321,28 @@ public class QueryParamBuilder {
   }
 
   /**
+   * Add common filter for set of Sentry users. This is used to simplify creating filters for
+   * privileges belonging to the specified set of users.
+   * @param query Query used for search
+   * @param paramBuilder paramBuilder for parameters
+   * @param userNames set of user names
+   * @return paramBuilder supplied or a new one if the supplied one is null.
+   */
+  public static QueryParamBuilder addUsersFilter(Query query, QueryParamBuilder paramBuilder,
+      Set<String> userNames) {
+    query.declareVariables(MSentryUser.class.getName() + " user");
+    if (paramBuilder == null) {
+      paramBuilder = new QueryParamBuilder();
+    }
+    if (userNames == null || userNames.isEmpty()) {
+      return paramBuilder;
+    }
+    paramBuilder.newChild().addSet("user.userName == ", userNames);
+    paramBuilder.addString("users.contains(user)");
+    return paramBuilder;
+  }
+
+  /**
    * Add multiple conditions for set of values.
    * <p>
    * Example:

http://git-wip-us.apache.org/repos/asf/sentry/blob/97f66634/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 56c506b..1c4bb37 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
@@ -65,9 +65,10 @@ import org.apache.sentry.provider.db.service.model.MSentryHmsNotification;
 import org.apache.sentry.provider.db.service.model.MSentryPathChange;
 import org.apache.sentry.provider.db.service.model.MSentryPermChange;
 import org.apache.sentry.provider.db.service.model.MSentryPrivilege;
+import org.apache.sentry.provider.db.service.model.MSentryGMPrivilege;
+import org.apache.sentry.provider.db.service.model.MSentryRole;
 import org.apache.sentry.provider.db.service.model.MSentryUser;
 import org.apache.sentry.provider.db.service.model.MSentryVersion;
-import org.apache.sentry.provider.db.service.model.MSentryRole;
 import org.apache.sentry.provider.db.service.model.MSentryUtil;
 import org.apache.sentry.provider.db.service.model.MPath;
 import org.apache.sentry.hdfs.service.thrift.TPrivilegeEntity;
@@ -81,6 +82,7 @@ import org.apache.sentry.api.service.thrift.TSentryMappingData;
 import org.apache.sentry.api.service.thrift.TSentryPrivilege;
 import org.apache.sentry.api.service.thrift.TSentryPrivilegeMap;
 import org.apache.sentry.api.service.thrift.TSentryRole;
+import org.apache.sentry.service.common.ServiceConstants.SentryEntityType;
 import org.apache.sentry.service.common.ServiceConstants.ServerConfig;
 import org.datanucleus.store.rdbms.exceptions.MissingTableException;
 import org.slf4j.Logger;
@@ -161,13 +163,16 @@ public class SentryStore {
   private static final String EMPTY_GRANTOR_PRINCIPAL = "--";
 
 
-  private static final Set<String> ALL_ACTIONS = Sets.newHashSet(AccessConstants.ALL,
+  private static final Set<String> ALL_ACTIONS = Sets.newHashSet(
+      AccessConstants.ALL, AccessConstants.ACTION_ALL,
       AccessConstants.SELECT, AccessConstants.INSERT, AccessConstants.ALTER,
       AccessConstants.CREATE, AccessConstants.DROP, AccessConstants.INDEX,
       AccessConstants.LOCK);
 
   // Now partial revoke just support action with SELECT,INSERT and ALL.
+  // Now partial revoke just support action with SELECT,INSERT, and ALL.
   // e.g. If we REVOKE SELECT from a privilege with action ALL, it will leads to INSERT
+  // e.g. If we REVOKE SELECT from a privilege with action ALL, it will leads to others individual
   // Otherwise, if we revoke other privilege(e.g. ALTER,DROP...), we will remove it from a role directly.
   private static final Set<String> PARTIAL_REVOKE_ACTIONS = Sets.newHashSet(AccessConstants.ALL,
       AccessConstants.ACTION_ALL.toLowerCase(), AccessConstants.SELECT, AccessConstants.INSERT);
@@ -343,6 +348,42 @@ public class SentryStore {
   }
 
   /**
+   * Get a single user with the given name inside a transaction
+   * @param pm Persistence Manager instance
+   * @param userName User name (should not be null)
+   * @return single user with the given name
+   */
+  public MSentryUser getUser(PersistenceManager pm, String userName) {
+    Query query = pm.newQuery(MSentryUser.class);
+    query.addExtension(LOAD_RESULTS_AT_COMMIT, "false");
+    query.setFilter("this.userName == :userName");
+    query.setUnique(true);
+    return (MSentryUser) query.execute(userName);
+  }
+
+  /**
+   * Create a sentry user and persist it. User name is the primary key for the
+   * user, so an attempt to create a user which exists fails with JDO exception.
+   *
+   * @param userName: Name of the user being persisted.
+   *    The name is normalized.
+   * @throws Exception
+   */
+  public void createSentryUser(final String userName) throws Exception {
+    tm.executeTransactionWithRetry(
+        pm -> {
+          pm.setDetachAllOnCommit(false); // No need to detach objects
+          String trimmedUserName = trimAndLower(userName);
+          if (getUser(pm, trimmedUserName) != null) {
+            throw new SentryAlreadyExistsException("User: " + trimmedUserName);
+          }
+          pm.makePersistent(
+              new MSentryUser(trimmedUserName, System.currentTimeMillis(), Sets.newHashSet()));
+          return null;
+        });
+  }
+
+  /**
    * Normalize the string values - remove leading and trailing whitespaces and
    * convert to lower case
    * @return normalized input
@@ -809,6 +850,369 @@ public class SentryStore {
   }
 
   /**
+   * Alter a given sentry user to grant a set of privileges.
+   * Internally calls alterSentryUserGrantPrivilege.
+   *
+   * @param grantorPrincipal User name
+   * @param userName User name
+   * @param privileges Set of privileges
+   * @throws Exception
+   */
+  public void alterSentryUserGrantPrivileges(final String grantorPrincipal,
+      final String userName, final Set<TSentryPrivilege> privileges) throws Exception {
+
+    try {
+      MSentryUser userEntry = getMSentryUserByName(userName, false);
+      if (userEntry == null) {
+        createSentryUser(userName);
+      }
+    } catch (SentryAlreadyExistsException e) {
+        // the user may be created by other thread, so swallow the exception and proceed
+    }
+
+    for (TSentryPrivilege privilege : privileges) {
+      alterSentryUserGrantPrivilege(grantorPrincipal, userName, privilege);
+    }
+  }
+
+  /**
+   * Alter a given sentry user to grant a privilege.
+   *
+   * @param grantorPrincipal User name
+   * @param userName the given user name
+   * @param privilege the given privilege
+   * @throws Exception
+   */
+  void alterSentryUserGrantPrivilege(final String grantorPrincipal,
+      final String userName, final TSentryPrivilege privilege) throws Exception {
+    tm.executeTransactionWithRetry(
+        new TransactionBlock<Object>() {
+          public Object execute(PersistenceManager pm) throws Exception {
+            pm.setDetachAllOnCommit(false); // No need to detach objects
+            String trimmedUserName = trimAndLower(userName);
+            // first do grant check
+            grantOptionCheck(pm, grantorPrincipal, privilege);
+
+            // Alter sentry User and grant Privilege.
+            MSentryPrivilege mPrivilege = alterSentryUserGrantPrivilegeCore(
+                pm, trimmedUserName, privilege);
+
+            if (mPrivilege != null) {
+              // update the privilege to be the one actually updated.
+              convertToTSentryPrivilege(mPrivilege, privilege);
+            }
+            return null;
+          }
+        });
+  }
+
+  /**
+   * Alter a given sentry user to grant a privilege, as well as persist the corresponding
+   * permission change to MSentryPermChange table in a single transaction.
+   *
+   * @param grantorPrincipal User name
+   * @param userName the given user name
+   * @param privilege the given privilege
+   * @param update the corresponding permission delta update.
+   * @throws Exception
+   *
+   */
+  synchronized void alterSentryUserGrantPrivilege(final String grantorPrincipal,
+      final String userName, final TSentryPrivilege privilege,
+      final Update update) throws Exception {
+
+    execute(update, new TransactionBlock<Object>() {
+      public Object execute(PersistenceManager pm) throws Exception {
+        pm.setDetachAllOnCommit(false); // No need to detach objects
+        String trimmedUserName = trimAndLower(userName);
+        // first do grant check
+        grantOptionCheck(pm, grantorPrincipal, privilege);
+
+        // Alter sentry User and grant Privilege.
+        MSentryPrivilege mPrivilege = alterSentryUserGrantPrivilegeCore(pm,
+            trimmedUserName, privilege);
+
+        if (mPrivilege != null) {
+          // update the privilege to be the one actually updated.
+          convertToTSentryPrivilege(mPrivilege, privilege);
+        }
+        return null;
+      }
+    });
+  }
+
+  /**
+   * Alter a given sentry user to grant a set of privileges, as well as persist the
+   * corresponding permission change to MSentryPermChange table in a single transaction.
+   * Internally calls alterSentryUserGrantPrivilege.
+   *
+   * @param grantorPrincipal User name
+   * @param userName the given user name
+   * @param privileges a Set of privileges
+   * @param privilegesUpdateMap the corresponding <privilege, DeltaTransactionBlock> map
+   * @throws Exception
+   *
+   */
+  public void alterSentryUserGrantPrivileges(final String grantorPrincipal,
+      final String userName, final Set<TSentryPrivilege> privileges,
+      final Map<TSentryPrivilege, Update> privilegesUpdateMap) throws Exception {
+
+    try {
+      MSentryUser userEntry = getMSentryUserByName(userName, false);
+      if (userEntry == null) {
+        createSentryUser(userName);
+      }
+    } catch (SentryAlreadyExistsException e) {
+      // the user may be created by other thread, so swallow the exception and proeed
+    }
+
+    Preconditions.checkNotNull(privilegesUpdateMap);
+    for (TSentryPrivilege privilege : privileges) {
+      Update update = privilegesUpdateMap.get(privilege);
+      if (update != null) {
+        alterSentryUserGrantPrivilege(grantorPrincipal, userName, privilege,
+            update);
+      } else {
+        alterSentryUserGrantPrivilege(grantorPrincipal, userName, privilege);
+      }
+    }
+  }
+
+  /**
+   * Get the user entry by user name
+   * @param userName the name of the user
+   * @return the user entry
+   * @throws Exception if the specified user does not exist
+   */
+  @VisibleForTesting
+  public MSentryUser getMSentryUserByName(final String userName) throws Exception {
+    return getMSentryUserByName(userName, true);
+  }
+
+  /**
+   * Get the user entry by user name
+   * @param userName the name of the user
+   * @param throwExceptionIfNotExist true: throw exception if user does not exist; false: return null
+   * @return the user entry or null
+   * @throws Exception if the specified user does not exist and throwExceptionIfNotExist is true
+   */
+  MSentryUser getMSentryUserByName(final String userName, boolean throwExceptionIfNotExist) throws Exception {
+    return tm.executeTransaction(
+        new TransactionBlock<MSentryUser>() {
+          public MSentryUser execute(PersistenceManager pm) throws Exception {
+            String trimmedUserName = trimAndLower(userName);
+            MSentryUser sentryUser = getUser(pm, trimmedUserName);
+            if (sentryUser == null) {
+              if (throwExceptionIfNotExist) {
+                throw noSuchUser(trimmedUserName);
+              }
+              else {
+                return null;
+              }
+            }
+            return sentryUser;
+          }
+        });
+  }
+
+  private MSentryPrivilege alterSentryUserGrantPrivilegeCore(PersistenceManager pm,
+      String userName, TSentryPrivilege privilege)
+      throws SentryNoSuchObjectException, SentryInvalidInputException {
+    MSentryPrivilege mPrivilege = null;
+    MSentryUser mUser = getUser(pm, userName);
+    if (mUser == null) {
+      throw noSuchUser(userName);
+    }
+
+    if(privilege.getPrivilegeScope().equalsIgnoreCase(PrivilegeScope.URI.name())
+        && StringUtils.isBlank(privilege.getURI())) {
+      throw new SentryInvalidInputException("cannot grant URI privileges to Null or EMPTY location");
+    }
+
+    if (!isNULL(privilege.getColumnName()) || !isNULL(privilege.getTableName())
+        || !isNULL(privilege.getDbName())) {
+      // If Grant is for ALL and Either INSERT/SELECT already exists..
+      // need to remove it and GRANT ALL..
+      if (AccessConstants.ALL.equalsIgnoreCase(privilege.getAction())
+          || AccessConstants.ACTION_ALL.equalsIgnoreCase(privilege.getAction())) {
+        TSentryPrivilege tNotAll = new TSentryPrivilege(privilege);
+        tNotAll.setAction(AccessConstants.SELECT);
+        MSentryPrivilege mSelect = getMSentryPrivilege(tNotAll, pm);
+        tNotAll.setAction(AccessConstants.INSERT);
+        MSentryPrivilege mInsert = getMSentryPrivilege(tNotAll, pm);
+        if ((mSelect != null) && mUser.getPrivileges().contains(mSelect)) {
+          mSelect.removeUser(mUser);
+          pm.makePersistent(mSelect);
+        }
+        if ((mInsert != null) && mUser.getPrivileges().contains(mInsert)) {
+          mInsert.removeUser(mUser);
+          pm.makePersistent(mInsert);
+        }
+      } else {
+        // If Grant is for Either INSERT/SELECT and ALL already exists..
+        // do nothing..
+        TSentryPrivilege tAll = new TSentryPrivilege(privilege);
+        tAll.setAction(AccessConstants.ALL);
+        MSentryPrivilege mAll1 = getMSentryPrivilege(tAll, pm);
+        tAll.setAction(AccessConstants.ACTION_ALL);
+        MSentryPrivilege mAll2 = getMSentryPrivilege(tAll, pm);
+        if (mAll1 != null && mUser.getPrivileges().contains(mAll1)) {
+          return null;
+        }
+        if (mAll2 != null && mUser.getPrivileges().contains(mAll2)) {
+          return null;
+        }
+      }
+    }
+
+    mPrivilege = getMSentryPrivilege(privilege, pm);
+    if (mPrivilege == null) {
+      mPrivilege = convertToMSentryPrivilege(privilege);
+    }
+    mPrivilege.appendUser(mUser);
+    pm.makePersistent(mPrivilege);
+    return mPrivilege;
+  }
+
+  /**
+   * Alter a given sentry user to revoke a privilege.
+   *
+   * @param grantorPrincipal User name
+   * @param userName the given user name
+   * @param tPrivilege the given privilege
+   * @throws Exception
+   *
+   */
+  void alterSentryUserRevokePrivilege(final String grantorPrincipal,
+      final String userName, final TSentryPrivilege tPrivilege) throws Exception {
+
+    tm.executeTransactionWithRetry(
+        new TransactionBlock<Object>() {
+          public Object execute(PersistenceManager pm) throws Exception {
+            pm.setDetachAllOnCommit(false); // No need to detach objects
+            String trimmedUserName = safeTrimLower(userName);
+            // first do revoke check
+            grantOptionCheck(pm, grantorPrincipal, tPrivilege);
+
+            alterSentryUserRevokePrivilegeCore(pm, trimmedUserName, tPrivilege);
+            return null;
+          }
+        });
+  }
+
+  /**
+   * Alter a given sentry user to revoke a set of privileges.
+   * Internally calls alterSentryUserRevokePrivilege.
+   *
+   * @param grantorPrincipal User name
+   * @param userName the given user name
+   * @param tPrivileges a Set of privileges
+   * @throws Exception
+   *
+   */
+  public void alterSentryUserRevokePrivileges(final String grantorPrincipal,
+      final String userName, final Set<TSentryPrivilege> tPrivileges) throws Exception {
+    for (TSentryPrivilege tPrivilege : tPrivileges) {
+      alterSentryUserRevokePrivilege(grantorPrincipal, userName, tPrivilege);
+    }
+  }
+
+  /**
+   * Alter a given sentry user to revoke a set of privileges, as well as persist the
+   * corresponding permission change to MSentryPermChange table in a single transaction.
+   * Internally calls alterSentryUserRevokePrivilege.
+   *
+   * @param grantorPrincipal User name
+   * @param userName the given user name
+   * @param tPrivileges a Set of privileges
+   * @param privilegesUpdateMap the corresponding <privilege, Update> map
+   * @throws Exception
+   *
+   */
+  public void alterSentryUserRevokePrivileges(final String grantorPrincipal,
+      final String userName, final Set<TSentryPrivilege> tPrivileges,
+      final Map<TSentryPrivilege, Update> privilegesUpdateMap)
+      throws Exception {
+
+    Preconditions.checkNotNull(privilegesUpdateMap);
+    for (TSentryPrivilege tPrivilege : tPrivileges) {
+      Update update = privilegesUpdateMap.get(tPrivilege);
+      if (update != null) {
+        alterSentryUserRevokePrivilege(grantorPrincipal, userName,
+            tPrivilege, update);
+      } else {
+        alterSentryUserRevokePrivilege(grantorPrincipal, userName,
+            tPrivilege);
+      }
+    }
+  }
+
+  /**
+   * Alter a given sentry user to revoke a privilege, as well as persist the corresponding
+   * permission change to MSentryPermChange table in a single transaction.
+   *
+   * @param grantorPrincipal User name
+   * @param userName the given user name
+   * @param tPrivilege the given privilege
+   * @param update the corresponding permission delta update transaction block
+   * @throws Exception
+   *
+   */
+  private synchronized void alterSentryUserRevokePrivilege(final String grantorPrincipal,
+      final String userName, final TSentryPrivilege tPrivilege,
+      final Update update) throws Exception {
+    execute(update, new TransactionBlock<Object>() {
+      public Object execute(PersistenceManager pm) throws Exception {
+        pm.setDetachAllOnCommit(false); // No need to detach objects
+        String trimmedUserName = safeTrimLower(userName);
+        // first do revoke check
+        grantOptionCheck(pm, grantorPrincipal, tPrivilege);
+
+        alterSentryUserRevokePrivilegeCore(pm, trimmedUserName, tPrivilege);
+        return null;
+      }
+    });
+  }
+
+  private void alterSentryUserRevokePrivilegeCore(PersistenceManager pm,
+      String userName, TSentryPrivilege tPrivilege)
+      throws SentryNoSuchObjectException, SentryInvalidInputException {
+    MSentryUser mUser = getUser(pm, userName);
+    if (mUser == null) {
+      throw noSuchUser(userName);
+    }
+    if(tPrivilege.getPrivilegeScope().equalsIgnoreCase(PrivilegeScope.URI.name())
+        && StringUtils.isBlank(tPrivilege.getURI())) {
+      throw new SentryInvalidInputException("cannot revoke URI privileges from Null or EMPTY location");
+    }
+
+    MSentryPrivilege mPrivilege = getMSentryPrivilege(tPrivilege, pm);
+    if (mPrivilege == null) {
+      mPrivilege = convertToMSentryPrivilege(tPrivilege);
+    } else {
+      mPrivilege = pm.detachCopy(mPrivilege);
+    }
+
+    Set<MSentryPrivilege> privilegeGraph = new HashSet<>();
+    if (mPrivilege.getGrantOption() != null) {
+      privilegeGraph.add(mPrivilege);
+    } else {
+      MSentryPrivilege mTure = new MSentryPrivilege(mPrivilege);
+      mTure.setGrantOption(true);
+      privilegeGraph.add(mTure);
+      MSentryPrivilege mFalse = new MSentryPrivilege(mPrivilege);
+      mFalse.setGrantOption(false);
+      privilegeGraph.add(mFalse);
+    }
+    // Get the privilege graph
+    populateChildren(pm, SentryEntityType.USER, Sets.newHashSet(userName), mPrivilege, privilegeGraph);
+    for (MSentryPrivilege childPriv : privilegeGraph) {
+      revokePrivilegeFromUser(pm, tPrivilege, mUser, childPriv);
+    }
+    pm.makePersistent(mUser);
+  }
+
+  /**
   * Alter a given sentry role to revoke a privilege.
   *
   * @param grantorPrincipal User name
@@ -935,7 +1339,7 @@ public class SentryStore {
       privilegeGraph.add(mFalse);
     }
     // Get the privilege graph
-    populateChildren(pm, Sets.newHashSet(roleName), mPrivilege, privilegeGraph);
+    populateChildren(pm, SentryEntityType.ROLE, Sets.newHashSet(roleName), mPrivilege, privilegeGraph);
     for (MSentryPrivilege childPriv : privilegeGraph) {
       revokePrivilegeFromRole(pm, tPrivilege, mRole, childPriv);
     }
@@ -948,7 +1352,8 @@ public class SentryStore {
    * privilege and add SELECT (INSERT was revoked) or INSERT (SELECT was revoked).
    */
   private void revokePartial(PersistenceManager pm,
-                             TSentryPrivilege requestedPrivToRevoke, MSentryRole mRole,
+                             TSentryPrivilege requestedPrivToRevoke,
+                             MSentryRole mRole, MSentryUser mUser,
                              MSentryPrivilege currentPrivilege) throws SentryInvalidInputException {
     MSentryPrivilege persistedPriv =
       getMSentryPrivilege(convertToTSentryPrivilege(currentPrivilege), pm);
@@ -959,34 +1364,69 @@ public class SentryStore {
       persistedPriv = convertToMSentryPrivilege(convertToTSentryPrivilege(currentPrivilege));
     }
 
-    if (requestedPrivToRevoke.getAction().equalsIgnoreCase("ALL") ||
-      requestedPrivToRevoke.getAction().equalsIgnoreCase("*")) {
+    if (requestedPrivToRevoke.getAction().equalsIgnoreCase(AccessConstants.ALL) ||
+      requestedPrivToRevoke.getAction().equalsIgnoreCase(AccessConstants.ACTION_ALL)) {
       if (!persistedPriv.getRoles().isEmpty()) {
-        persistedPriv.removeRole(mRole);
-        if (persistedPriv.getRoles().isEmpty()) {
+        if (mRole != null) {
+          persistedPriv.removeRole(mRole);
+        }
+        if (mUser != null) {
+          persistedPriv.removeUser(mUser);
+        }
+
+        if (isPrivilegeStall(persistedPriv)) {
           pm.deletePersistent(persistedPriv);
         } else {
           pm.makePersistent(persistedPriv);
         }
       }
-    } else if (requestedPrivToRevoke.getAction().equalsIgnoreCase(AccessConstants.SELECT)
-      && !currentPrivilege.getAction().equalsIgnoreCase(AccessConstants.INSERT)) {
-      revokeRolePartial(pm, mRole, currentPrivilege, persistedPriv, AccessConstants.INSERT);
-    } else if (requestedPrivToRevoke.getAction().equalsIgnoreCase(AccessConstants.INSERT)
-      && !currentPrivilege.getAction().equalsIgnoreCase(AccessConstants.SELECT)) {
-      revokeRolePartial(pm, mRole, currentPrivilege, persistedPriv, AccessConstants.SELECT);
+    } else {
+
+      Set<String> addActions = new HashSet<String>();
+      for (String actionToAdd : PARTIAL_REVOKE_ACTIONS) {
+        if( !requestedPrivToRevoke.getAction().equalsIgnoreCase(actionToAdd) &&
+            !currentPrivilege.getAction().equalsIgnoreCase(actionToAdd) &&
+            !AccessConstants.ALL.equalsIgnoreCase(actionToAdd) &&
+            !AccessConstants.ACTION_ALL.equalsIgnoreCase(actionToAdd)) {
+          addActions.add(actionToAdd);
+        }
+      }
+
+      if (mRole != null) {
+        revokeRolePartial(pm, mRole, currentPrivilege, persistedPriv, addActions);
+      }
+
+      if (mUser != null) {
+        revokeUserPartial(pm, mUser, currentPrivilege, persistedPriv, addActions);
+      }
     }
   }
 
+  private boolean isPrivilegeStall(MSentryPrivilege privilege) {
+    if (privilege.getUsers().isEmpty() && privilege.getRoles().isEmpty()) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean isPrivilegeStall(MSentryGMPrivilege privilege) {
+    if (privilege.getRoles().isEmpty()) {
+      return true;
+    }
+
+    return false;
+  }
+
   private void revokeRolePartial(PersistenceManager pm, MSentryRole mRole,
                                  MSentryPrivilege currentPrivilege,
                                  MSentryPrivilege persistedPriv,
-                                 String addAction) throws SentryInvalidInputException {
+                                 Set<String> addActions) throws SentryInvalidInputException {
     // If table / URI, remove ALL
     persistedPriv = getMSentryPrivilege(convertToTSentryPrivilege(persistedPriv), pm);
     if (persistedPriv != null && !persistedPriv.getRoles().isEmpty()) {
       persistedPriv.removeRole(mRole);
-      if (persistedPriv.getRoles().isEmpty()) {
+      if (isPrivilegeStall(persistedPriv)) {
         pm.deletePersistent(persistedPriv);
       } else {
         pm.makePersistent(persistedPriv);
@@ -996,15 +1436,20 @@ public class SentryStore {
     persistedPriv = getMSentryPrivilege(convertToTSentryPrivilege(currentPrivilege), pm);
     if (persistedPriv != null && mRole.getPrivileges().contains(persistedPriv)) {
       persistedPriv.removeRole(mRole);
-      if (persistedPriv.getRoles().isEmpty()) {
+      if (isPrivilegeStall(persistedPriv)) {
         pm.deletePersistent(persistedPriv);
       } else {
         pm.makePersistent(persistedPriv);
       }
-      currentPrivilege.setAction(addAction);
-      persistedPriv = getMSentryPrivilege(convertToTSentryPrivilege(currentPrivilege), pm);
-      if (persistedPriv == null) {
-        persistedPriv = convertToMSentryPrivilege(convertToTSentryPrivilege(currentPrivilege));
+
+      // add decomposted actions
+      for (String addAction : addActions) {
+        currentPrivilege.setAction(addAction);
+        TSentryPrivilege tSentryPrivilege = convertToTSentryPrivilege(currentPrivilege);
+        persistedPriv = getMSentryPrivilege(tSentryPrivilege, pm);
+        if (persistedPriv == null) {
+          persistedPriv = convertToMSentryPrivilege(tSentryPrivilege);
+        }
         mRole.appendPrivilege(persistedPriv);
       }
       persistedPriv.appendRole(mRole);
@@ -1012,6 +1457,45 @@ public class SentryStore {
     }
   }
 
+  private void revokeUserPartial(PersistenceManager pm, MSentryUser mUser,
+      MSentryPrivilege currentPrivilege,
+      MSentryPrivilege persistedPriv,
+      Set<String> addActions) throws SentryInvalidInputException {
+    // If table / URI, remove ALL
+    persistedPriv = getMSentryPrivilege(convertToTSentryPrivilege(persistedPriv), pm);
+    if (persistedPriv != null && !persistedPriv.getUsers().isEmpty()) {
+      persistedPriv.removeUser(mUser);
+      if (isPrivilegeStall(persistedPriv)) {
+        pm.deletePersistent(persistedPriv);
+      } else {
+        pm.makePersistent(persistedPriv);
+      }
+    }
+    currentPrivilege.setAction(AccessConstants.ALL);
+    persistedPriv = getMSentryPrivilege(convertToTSentryPrivilege(currentPrivilege), pm);
+    if (persistedPriv != null && mUser.getPrivileges().contains(persistedPriv)) {
+      persistedPriv.removeUser(mUser);
+      if (isPrivilegeStall(persistedPriv)) {
+        pm.deletePersistent(persistedPriv);
+      } else {
+        pm.makePersistent(persistedPriv);
+      }
+
+      // add decomposted actions
+      for (String addAction : addActions) {
+        currentPrivilege.setAction(addAction);
+        TSentryPrivilege tSentryPrivilege = convertToTSentryPrivilege(currentPrivilege);
+        persistedPriv = getMSentryPrivilege(tSentryPrivilege, pm);
+        if (persistedPriv == null) {
+          persistedPriv = convertToMSentryPrivilege(tSentryPrivilege);
+        }
+        mUser.appendPrivilege(persistedPriv);
+      }
+      persistedPriv.appendUser(mUser);
+      pm.makePersistent(persistedPriv);
+    }
+  }
+
   /**
    * Revoke privilege from role
    */
@@ -1019,16 +1503,41 @@ public class SentryStore {
                                        MSentryRole mRole, MSentryPrivilege mPrivilege)
     throws SentryInvalidInputException {
     if (PARTIAL_REVOKE_ACTIONS.contains(mPrivilege.getAction())) {
-      // if this privilege is in {ALL,SELECT,INSERT}
+      // if this privilege is in parital revoke actions
       // we will do partial revoke
-      revokePartial(pm, tPrivilege, mRole, mPrivilege);
+      revokePartial(pm, tPrivilege, mRole, null, mPrivilege);
     } else {
-      // if this privilege is not ALL, SELECT nor INSERT,
+      // otherwise,
       // we will revoke it from role directly
       MSentryPrivilege persistedPriv = getMSentryPrivilege(convertToTSentryPrivilege(mPrivilege), pm);
       if (persistedPriv != null && !persistedPriv.getRoles().isEmpty()) {
         persistedPriv.removeRole(mRole);
-        if (persistedPriv.getRoles().isEmpty()) {
+        if (isPrivilegeStall(persistedPriv)) {
+          pm.deletePersistent(persistedPriv);
+        } else {
+          pm.makePersistent(persistedPriv);
+        }
+      }
+    }
+  }
+
+  /**
+   * Revoke privilege from user
+   */
+  private void revokePrivilegeFromUser(PersistenceManager pm, TSentryPrivilege tPrivilege,
+      MSentryUser mUser, MSentryPrivilege mPrivilege)
+      throws SentryInvalidInputException {
+    if (PARTIAL_REVOKE_ACTIONS.contains(mPrivilege.getAction())) {
+      // if this privilege is in parital revoke actions
+      // we will do partial revoke
+      revokePartial(pm, tPrivilege, null, mUser, mPrivilege);
+    } else {
+      // otherwise,
+      // we will revoke it from user directly
+      MSentryPrivilege persistedPriv = getMSentryPrivilege(convertToTSentryPrivilege(mPrivilege), pm);
+      if (persistedPriv != null && !persistedPriv.getUsers().isEmpty()) {
+        persistedPriv.removeUser(mUser);
+        if (isPrivilegeStall(persistedPriv)) {
           pm.deletePersistent(persistedPriv);
         } else {
           pm.makePersistent(persistedPriv);
@@ -1041,26 +1550,26 @@ public class SentryStore {
    * Explore Privilege graph and collect child privileges.
    * The responsibility to commit/rollback the transaction should be handled by the caller.
    */
-  private void populateChildren(PersistenceManager pm, Set<String> roleNames, MSentryPrivilege priv,
+  private void populateChildren(PersistenceManager pm, SentryEntityType entityType, Set<String> entityNames, MSentryPrivilege priv,
       Collection<MSentryPrivilege> children) throws SentryInvalidInputException {
     Preconditions.checkNotNull(pm);
     if (!isNULL(priv.getServerName()) || !isNULL(priv.getDbName())
         || !isNULL(priv.getTableName())) {
       // Get all TableLevel Privs
-      Set<MSentryPrivilege> childPrivs = getChildPrivileges(pm, roleNames, priv);
+      Set<MSentryPrivilege> childPrivs = getChildPrivileges(pm, entityType, entityNames, priv);
       for (MSentryPrivilege childPriv : childPrivs) {
         // Only recurse for table level privs..
         if (!isNULL(childPriv.getDbName()) && !isNULL(childPriv.getTableName())
             && !isNULL(childPriv.getColumnName())) {
-          populateChildren(pm, roleNames, childPriv, children);
+          populateChildren(pm, entityType, entityNames, childPriv, children);
         }
         // The method getChildPrivileges() didn't do filter on "action",
         // if the action is not "All", it should judge the action of children privilege.
         // For example: a user has a privilege “All on Col1”,
         // if the operation is “REVOKE INSERT on table”
         // the privilege should be the child of table level privilege.
-        // but the privilege may still have other meaning, likes "SELECT on Col1".
-        // and the privileges like "SELECT on Col1" should not be revoke.
+        // but the privilege may still have other meaning, likes "SELECT, CREATE etc. on Col1".
+        // and the privileges like "SELECT, CREATE etc. on Col1" should not be revoke.
         if (!priv.isActionALL()) {
           if (childPriv.isActionALL()) {
             // If the child privilege is All, we should convert it to the same
@@ -1077,7 +1586,7 @@ public class SentryStore {
     }
   }
 
-  private Set<MSentryPrivilege> getChildPrivileges(PersistenceManager pm, Set<String> roleNames,
+  private Set<MSentryPrivilege> getChildPrivileges(PersistenceManager pm, SentryEntityType entityType, Set<String> entityNames,
       MSentryPrivilege parent) throws SentryInvalidInputException {
     // Column and URI do not have children
     if (!isNULL(parent.getColumnName()) || !isNULL(parent.getURI())) {
@@ -1085,29 +1594,35 @@ public class SentryStore {
     }
 
     Query query = pm.newQuery(MSentryPrivilege.class);
-    QueryParamBuilder paramBuilder = QueryParamBuilder.addRolesFilter(query, null, roleNames)
-            .add(SERVER_NAME, parent.getServerName());
+    QueryParamBuilder paramBuilder = null;
+    if (entityType == SentryEntityType.ROLE) {
+      paramBuilder = QueryParamBuilder.addRolesFilter(query, null, entityNames).add(SERVER_NAME, parent.getServerName());
+    } else if (entityType == SentryEntityType.USER) {
+      paramBuilder = QueryParamBuilder.addUsersFilter(query, null, entityNames).add(SERVER_NAME, parent.getServerName());
+    } else {
+      throw new SentryInvalidInputException("entityType" + entityType + " is not valid");
+    }
 
     if (!isNULL(parent.getDbName())) {
       paramBuilder.add(DB_NAME, parent.getDbName());
       if (!isNULL(parent.getTableName())) {
         paramBuilder.add(TABLE_NAME, parent.getTableName())
-                .addNotNull(COLUMN_NAME);
+            .addNotNull(COLUMN_NAME);
       } else {
         paramBuilder.addNotNull(TABLE_NAME);
       }
     } else {
-      // Andd condition dbName != NULL || URI != NULL
+      // Add condition dbName != NULL || URI != NULL
       paramBuilder.newChild()
-              .addNotNull(DB_NAME)
-              .addNotNull(URI);
+          .addNotNull(DB_NAME)
+          .addNotNull(URI);
     }
 
     query.setFilter(paramBuilder.toString());
     query.setResult("privilegeScope, serverName, dbName, tableName, columnName," +
         " URI, action, grantOption");
     List<Object[]> privObjects =
-            (List<Object[]>) query.executeWithMap(paramBuilder.getArguments());
+        (List<Object[]>) query.executeWithMap(paramBuilder.getArguments());
     Set<MSentryPrivilege> privileges = new HashSet<>(privObjects.size());
     for (Object[] privObj : privObjects) {
       String scope        = (String)privObj[0];
@@ -1119,13 +1634,76 @@ public class SentryStore {
       String action       = (String) privObj[6];
       Boolean grantOption = (Boolean) privObj[7];
       MSentryPrivilege priv =
-              new MSentryPrivilege(scope, serverName, dbName, tableName,
-                      columnName, URI, action, grantOption);
+          new MSentryPrivilege(scope, serverName, dbName, tableName,
+              columnName, URI, action, grantOption);
       privileges.add(priv);
     }
     return privileges;
   }
 
+  /**
+   * Drop a given sentry user.
+   *
+   * @param userName the given user name
+   * @throws Exception
+   */
+  public void dropSentryUser(final String userName) throws Exception {
+    tm.executeTransactionWithRetry(
+        new TransactionBlock<Object>() {
+          public Object execute(PersistenceManager pm) throws Exception {
+            pm.setDetachAllOnCommit(false); // No need to detach objects
+            dropSentryUserCore(pm, userName);
+            return null;
+          }
+        });
+  }
+
+  /**
+   * Drop a given sentry user. As well as persist the corresponding
+   * permission change to MSentryPermChange table in a single transaction.
+   *
+   * @param userName the given user name
+   * @param update the corresponding permission delta update
+   * @throws Exception
+   */
+  public synchronized void dropSentryUser(final String userName,
+      final Update update) throws Exception {
+    execute(update, new TransactionBlock<Object>() {
+      public Object execute(PersistenceManager pm) throws Exception {
+        pm.setDetachAllOnCommit(false); // No need to detach objects
+        dropSentryUserCore(pm, userName);
+        return null;
+      }
+    });
+  }
+
+  private void dropSentryUserCore(PersistenceManager pm, String userName)
+      throws SentryNoSuchObjectException {
+    String lUserName = trimAndLower(userName);
+    MSentryUser sentryUser = getUser(pm, lUserName);
+    if (sentryUser == null) {
+      throw noSuchUser(lUserName);
+    }
+    removePrivilegesForUser(pm, sentryUser);
+    pm.deletePersistent(sentryUser);
+  }
+
+  /**
+   * Removes all the privileges associated with
+   * a particular user. After this dis-association if the
+   * privilege doesn't have any users associated it will be
+   * removed from the underlying persistence layer.
+   * @param pm Instance of PersistenceManager
+   * @param sentryUser User for which all the privileges are to be removed.
+   */
+  private void removePrivilegesForUser(PersistenceManager pm, MSentryUser sentryUser) {
+    List<MSentryPrivilege> privilegesCopy = new ArrayList<>(sentryUser.getPrivileges());
+
+    sentryUser.removePrivileges();
+
+    removeStaledPrivileges(pm, privilegesCopy);
+  }
+
   @SuppressWarnings("unchecked")
   private List<MSentryPrivilege> getMSentryPrivileges(TSentryPrivilege tPriv, PersistenceManager pm) {
     Query query = pm.newQuery(MSentryPrivilege.class);
@@ -1227,13 +1805,32 @@ public class SentryStore {
    */
   private void removePrivileges(PersistenceManager pm, MSentryRole sentryRole) {
     List<MSentryPrivilege> privilegesCopy = new ArrayList<>(sentryRole.getPrivileges());
-    List<MSentryPrivilege> stalePrivileges = new ArrayList<>(0);
+    List<MSentryGMPrivilege> gmPrivilegesCopy = new ArrayList<>(sentryRole.getGmPrivileges());
 
     sentryRole.removePrivileges();
     // with SENTRY-398 generic model
     sentryRole.removeGMPrivileges();
+
+    removeStaledPrivileges(pm, privilegesCopy);
+    removeStaledGMPrivileges(pm, gmPrivilegesCopy);
+  }
+
+  private void removeStaledPrivileges(PersistenceManager pm, List<MSentryPrivilege> privilegesCopy) {
+    List<MSentryPrivilege> stalePrivileges = new ArrayList<>(0);
     for (MSentryPrivilege privilege : privilegesCopy) {
-      if(privilege.getRoles().isEmpty()) {
+      if (isPrivilegeStall(privilege)) {
+        stalePrivileges.add(privilege);
+      }
+    }
+    if(!stalePrivileges.isEmpty()) {
+      pm.deletePersistentAll(stalePrivileges);
+    }
+  }
+
+  private void removeStaledGMPrivileges(PersistenceManager pm, List<MSentryGMPrivilege> privilegesCopy) {
+    List<MSentryGMPrivilege> stalePrivileges = new ArrayList<>(0);
+    for (MSentryGMPrivilege privilege : privilegesCopy) {
+      if (isPrivilegeStall(privilege)) {
         stalePrivileges.add(privilege);
       }
     }
@@ -1241,6 +1838,7 @@ public class SentryStore {
       pm.deletePersistentAll(stalePrivileges);
     }
   }
+
   /**
    * Assign a given role to a set of groups.
    *
@@ -1510,75 +2108,95 @@ public class SentryStore {
             });
   }
 
-  private List<MSentryPrivilege> getMSentryPrivileges(final Set<String> roleNames,
-                                                      final TSentryAuthorizable
-                                                              authHierarchy)
-          throws Exception {
-    if (roleNames == null || roleNames.isEmpty()) {
+  private List<MSentryPrivilege> getMSentryPrivileges(final SentryEntityType entityType, final Set<String> entityNames,
+      final TSentryAuthorizable authHierarchy)
+      throws Exception {
+    if (entityNames == null || entityNames.isEmpty()) {
       return Collections.emptyList();
     }
 
     return tm.executeTransaction(
-            pm -> {
-              Query query = pm.newQuery(MSentryPrivilege.class);
-              QueryParamBuilder paramBuilder = QueryParamBuilder.addRolesFilter(query, null, roleNames);
+        pm -> {
+          Query query = pm.newQuery(MSentryPrivilege.class);
+          QueryParamBuilder paramBuilder = null;
+          if (entityType == SentryEntityType.ROLE) {
+            paramBuilder = QueryParamBuilder.addRolesFilter(query, null, entityNames);
+          } else if (entityType == SentryEntityType.USER) {
+            paramBuilder = QueryParamBuilder.addUsersFilter(query, null, entityNames);
+          } else {
+            throw new SentryInvalidInputException("entityType" + entityType + " is not valid");
+          }
 
-              if (authHierarchy != null && authHierarchy.getServer() != null) {
-                paramBuilder.add(SERVER_NAME, authHierarchy.getServer());
-                if (authHierarchy.getDb() != null) {
+          if (authHierarchy != null && authHierarchy.getServer() != null) {
+            paramBuilder.add(SERVER_NAME, authHierarchy.getServer());
+            if (authHierarchy.getDb() != null) {
+              paramBuilder.addNull(URI)
+                  .newChild()
+                  .add(DB_NAME, authHierarchy.getDb())
+                  .addNull(DB_NAME);
+              if (authHierarchy.getTable() != null
+                  && !AccessConstants.ALL.equalsIgnoreCase(authHierarchy.getTable())) {
+                if (!AccessConstants.SOME.equalsIgnoreCase(authHierarchy.getTable())) {
                   paramBuilder.addNull(URI)
-                          .newChild()
-                            .add(DB_NAME, authHierarchy.getDb())
-                            .addNull(DB_NAME);
-                  if (authHierarchy.getTable() != null
-                      && !AccessConstants.ALL.equalsIgnoreCase(authHierarchy.getTable())) {
-                    if (!AccessConstants.SOME.equalsIgnoreCase(authHierarchy.getTable())) {
-                      paramBuilder.addNull(URI)
-                              .newChild()
-                                .add(TABLE_NAME, authHierarchy.getTable())
-                                .addNull(TABLE_NAME);
-                    }
-                    if (authHierarchy.getColumn() != null
-                        && !AccessConstants.ALL.equalsIgnoreCase(authHierarchy.getColumn())
-                        && !AccessConstants.SOME.equalsIgnoreCase(authHierarchy.getColumn())) {
-                      paramBuilder.addNull(URI)
-                              .newChild()
-                                .add(COLUMN_NAME, authHierarchy.getColumn())
-                                .addNull(COLUMN_NAME);
-                    }
-                  }
+                      .newChild()
+                      .add(TABLE_NAME, authHierarchy.getTable())
+                      .addNull(TABLE_NAME);
                 }
-                if (authHierarchy.getUri() != null) {
-                  paramBuilder.addNull(DB_NAME)
-                          .newChild()
-                            .addNull(URI)
-                            .newChild()
-                              .addNotNull(URI)
-                              .addCustomParam("\"" + authHierarchy.getUri() +
-                                      "\".startsWith(:URI)", URI, authHierarchy.getUri());
+                if (authHierarchy.getColumn() != null
+                    && !AccessConstants.ALL.equalsIgnoreCase(authHierarchy.getColumn())
+                    && !AccessConstants.SOME.equalsIgnoreCase(authHierarchy.getColumn())) {
+                  paramBuilder.addNull(URI)
+                      .newChild()
+                      .add(COLUMN_NAME, authHierarchy.getColumn())
+                      .addNull(COLUMN_NAME);
                 }
               }
+            }
+            if (authHierarchy.getUri() != null) {
+              paramBuilder.addNull(DB_NAME)
+                  .newChild()
+                  .addNull(URI)
+                  .newChild()
+                  .addNotNull(URI)
+                  .addCustomParam("\"" + authHierarchy.getUri() +
+                      "\".startsWith(:URI)", URI, authHierarchy.getUri());
+            }
+          }
 
-              query.setFilter(paramBuilder.toString());
-              @SuppressWarnings("unchecked")
-              List<MSentryPrivilege> result =
-                      (List<MSentryPrivilege>)
-                              query.executeWithMap(paramBuilder.getArguments());
-              return result;
-            });
+          query.setFilter(paramBuilder.toString());
+          @SuppressWarnings("unchecked")
+          List<MSentryPrivilege> result =
+              (List<MSentryPrivilege>)
+                  query.executeWithMap(paramBuilder.getArguments());
+          return result;
+        });
   }
 
-  private List<MSentryPrivilege> getMSentryPrivilegesByAuth(final Set<String> roleNames,
-                                                            final TSentryAuthorizable
-                                                                    authHierarchy) throws Exception {
+  private List<MSentryPrivilege> getMSentryPrivilegesByAuth(
+      final SentryEntityType entityType,
+      final Set<String> entityNames,
+      final TSentryAuthorizable
+      authHierarchy) throws Exception {
       return tm.executeTransaction(
               pm -> {
                 Query query = pm.newQuery(MSentryPrivilege.class);
                 QueryParamBuilder paramBuilder = newQueryParamBuilder();
-                if (roleNames == null || roleNames.isEmpty()) {
-                  paramBuilder.addString("!roles.isEmpty()");
+                if (entityNames == null || entityNames.isEmpty()) {
+                  if (entityType == SentryEntityType.ROLE) {
+                    paramBuilder.addString("!roles.isEmpty()");
+                  } else if (entityType == SentryEntityType.USER) {
+                    paramBuilder.addString("!users.isEmpty()");
+                  } else {
+                    throw new SentryInvalidInputException("entityType: " + entityType + " is invalid");
+                  }
                 } else {
-                  QueryParamBuilder.addRolesFilter(query, paramBuilder, roleNames);
+                  if (entityType == SentryEntityType.ROLE) {
+                    QueryParamBuilder.addRolesFilter(query, paramBuilder, entityNames);
+                  } else if (entityType == SentryEntityType.USER) {
+                    QueryParamBuilder.addUsersFilter(query, paramBuilder, entityNames);
+                  } else {
+                    throw new SentryInvalidInputException("entityType" + entityType + " is not valid");
+                  }
                 }
                 if (authHierarchy.getServer() != null) {
                   paramBuilder.add(SERVER_NAME, authHierarchy.getServer());
@@ -1612,10 +2230,37 @@ public class SentryStore {
               });
   }
 
+  private Set<MSentryPrivilege> getMSentryPrivilegesByUserName(String userName)
+      throws Exception {
+    MSentryUser mSentryUser = getMSentryUserByName(userName);
+    return mSentryUser.getPrivileges();
+  }
+
+  /**
+   * Gets sentry privilege objects for a given userName from the persistence layer
+   * @param userName : userName to look up
+   * @return : Set of thrift sentry privilege objects
+   * @throws Exception
+   */
+
+  public Set<TSentryPrivilege> getAllTSentryPrivilegesByUserName(String userName)
+      throws Exception {
+    return convertToTSentryPrivileges(getMSentryPrivilegesByUserName(userName));
+  }
+
+  /**
+   * Get all privileges associated with the authorizable and roles from input roles or input groups
+   * @param groups the groups to get roles, then get their privileges
+   * @param activeRoles the roles to get privileges
+   * @param authHierarchy the authorizables
+   * @param isAdmin true: user is admin; false: is not admin
+   * @return the privilege map. The key is role name
+   * @throws Exception
+   */
   public TSentryPrivilegeMap listSentryPrivilegesByAuthorizable(Set<String> groups,
       TSentryActiveRoleSet activeRoles,
       TSentryAuthorizable authHierarchy, boolean isAdmin)
-          throws Exception {
+      throws Exception {
     Map<String, Set<TSentryPrivilege>> resultPrivilegeMap = Maps.newTreeMap();
     Set<String> roles = getRolesToQuery(groups, null, new TSentryActiveRoleSet(true, null));
 
@@ -1629,8 +2274,8 @@ public class SentryStore {
     // An empty 'roles' is a treated as a wildcard (in case of admin role)..
     // so if not admin, don't return anything if 'roles' is empty..
     if (isAdmin || !roles.isEmpty()) {
-      List<MSentryPrivilege> mSentryPrivileges = getMSentryPrivilegesByAuth(roles,
-          authHierarchy);
+      List<MSentryPrivilege> mSentryPrivileges =
+          getMSentryPrivilegesByAuth(SentryEntityType.ROLE, roles, authHierarchy);
       for (MSentryPrivilege priv : mSentryPrivileges) {
         for (MSentryRole role : priv.getRoles()) {
           TSentryPrivilege tPriv = convertToTSentryPrivilege(priv);
@@ -1647,6 +2292,41 @@ public class SentryStore {
     return new TSentryPrivilegeMap(resultPrivilegeMap);
   }
 
+  /**
+   * Get all privileges associated with the authorizable and input users
+   * @param userNames the users to get their privileges
+   * @param authHierarchy the authorizables
+   * @param isAdmin true: user is admin; false: is not admin
+   * @return the privilege map. The key is user name
+   * @throws Exception
+   */
+  public TSentryPrivilegeMap listSentryPrivilegesByAuthorizableForUser(Set<String> userNames,
+      TSentryAuthorizable authHierarchy, boolean isAdmin)
+      throws Exception {
+    Map<String, Set<TSentryPrivilege>> resultPrivilegeMap = Maps.newTreeMap();
+
+    // An empty 'userNames' is a treated as a wildcard (in case of admin role)..
+    // so if not admin, don't return anything if 'roles' is empty..
+    if (isAdmin || ((userNames != null) && (!userNames.isEmpty()))) {
+      List<MSentryPrivilege> mSentryPrivileges =
+          getMSentryPrivilegesByAuth(SentryEntityType.USER, userNames, authHierarchy);
+      for (MSentryPrivilege priv : mSentryPrivileges) {
+        for (MSentryUser user : priv.getUsers()) {
+          TSentryPrivilege tPriv = convertToTSentryPrivilege(priv);
+          if (resultPrivilegeMap.containsKey(user.getUserName())) {
+            resultPrivilegeMap.get(user.getUserName()).add(tPriv);
+          } else {
+            Set<TSentryPrivilege> tPrivSet = Sets.newTreeSet();
+            tPrivSet.add(tPriv);
+            resultPrivilegeMap.put(user.getUserName(), tPrivSet);
+          }
+        }
+      }
+    }
+    return new TSentryPrivilegeMap(resultPrivilegeMap);
+  }
+
+
   private Set<MSentryPrivilege> getMSentryPrivilegesByRoleName(String roleName)
       throws Exception {
     MSentryRole mSentryRole = getMSentryRoleByName(roleName);
@@ -1668,13 +2348,14 @@ public class SentryStore {
 
   /**
    * Gets sentry privilege objects for criteria from the persistence layer
-   * @param roleNames : roleNames to look up (required)
+   * @param entityType : the type of the entity (required)
+   * @param entityNames : entity names to look up (required)
    * @param authHierarchy : filter push down based on auth hierarchy (optional)
    * @return : Set of thrift sentry privilege objects
    * @throws SentryInvalidInputException
    */
 
-  public Set<TSentryPrivilege> getTSentryPrivileges(Set<String> roleNames,
+  public Set<TSentryPrivilege> getTSentryPrivileges(SentryEntityType entityType, Set<String> entityNames,
                                                     TSentryAuthorizable authHierarchy)
           throws Exception {
     if (authHierarchy.getServer() == null) {
@@ -1689,7 +2370,7 @@ public class SentryStore {
     if (authHierarchy.getUri() == null && authHierarchy.getDb() == null) {
       throw new SentryInvalidInputException("One of uri or dbName must not be null !!");
     }
-    return convertToTSentryPrivileges(getMSentryPrivileges(roleNames, authHierarchy));
+    return convertToTSentryPrivileges(getMSentryPrivileges(entityType, entityNames, authHierarchy));
   }
 
   /**
@@ -1852,7 +2533,12 @@ public class SentryStore {
       TSentryActiveRoleSet roleSet, TSentryAuthorizable authHierarchy) throws Exception {
     Set<String> result = Sets.newHashSet();
     Set<String> rolesToQuery = getRolesToQuery(groups, users, roleSet);
-    List<MSentryPrivilege> mSentryPrivileges = getMSentryPrivileges(rolesToQuery, authHierarchy);
+    List<MSentryPrivilege> mSentryPrivileges = getMSentryPrivileges(SentryEntityType.ROLE, rolesToQuery, authHierarchy);
+    for (MSentryPrivilege priv : mSentryPrivileges) {
+      result.add(toAuthorizable(priv));
+    }
+
+    mSentryPrivileges = getMSentryPrivileges(SentryEntityType.USER, users, authHierarchy);
     for (MSentryPrivilege priv : mSentryPrivileges) {
       result.add(toAuthorizable(priv));
     }
@@ -1863,7 +2549,31 @@ public class SentryStore {
   public boolean hasAnyServerPrivileges(Set<String> groups, Set<String> users,
       TSentryActiveRoleSet roleSet, String server) throws Exception {
     Set<String> rolesToQuery = getRolesToQuery(groups, users, roleSet);
-    return hasAnyServerPrivileges(rolesToQuery, server);
+    if (hasAnyServerPrivileges(rolesToQuery, server)) {
+      return true;
+    }
+
+    return hasAnyServerPrivilegesForUser(users, server);
+  }
+
+  private boolean hasAnyServerPrivilegesForUser(final Set<String> userNames, final String serverName) throws Exception {
+    if (userNames == null || userNames.isEmpty()) {
+      return false;
+    }
+    return tm.executeTransaction(
+        new TransactionBlock<Boolean>() {
+          public Boolean execute(PersistenceManager pm) throws Exception {
+            pm.setDetachAllOnCommit(false); // No need to detach objects
+            Query query = pm.newQuery(MSentryPrivilege.class);
+            query.addExtension(LOAD_RESULTS_AT_COMMIT, "false");
+            QueryParamBuilder paramBuilder = QueryParamBuilder.addUsersFilter(query,null, userNames);
+            paramBuilder.add(SERVER_NAME, serverName);
+            query.setFilter(paramBuilder.toString());
+            query.setResult("count(this)");
+            Long numPrivs = (Long) query.executeWithMap(paramBuilder.getArguments());
+            return numPrivs > 0;
+          }
+        });
   }
 
   private Set<String> getRolesToQuery(final Set<String> groups, final Set<String> users,
@@ -2303,9 +3013,9 @@ public class SentryStore {
       Collection<MSentryPrivilege> privilegeGraph = new HashSet<>();
       if (parent != null) {
         privilegeGraph.add(parent);
-        populateChildren(pm, Sets.newHashSet(role.getRoleName()), parent, privilegeGraph);
+        populateChildren(pm, SentryEntityType.ROLE, Sets.newHashSet(role.getRoleName()), parent, privilegeGraph);
       } else {
-        populateChildren(pm, Sets.newHashSet(role.getRoleName()), convertToMSentryPrivilege(tPrivilege),
+        populateChildren(pm, SentryEntityType.ROLE, Sets.newHashSet(role.getRoleName()), convertToMSentryPrivilege(tPrivilege),
           privilegeGraph);
       }
       // 2. revoke privilege and child privileges
@@ -3647,6 +4357,15 @@ public class SentryStore {
   }
 
   /**
+   * Return exception for nonexistent user
+   * @param userName User name
+   * @return SentryNoSuchObjectException with appropriate message
+   */
+  private static SentryNoSuchObjectException noSuchUser(String userName) {
+    return new SentryNoSuchObjectException("nonexistent user " + userName);
+  }
+
+  /**
    * Return exception for nonexistent group
    * @param groupName Group name
    * @return SentryNoSuchObjectException with appropriate message

http://git-wip-us.apache.org/repos/asf/sentry/blob/97f66634/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 0322cc3..e2d24e5 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
@@ -70,9 +70,11 @@ import org.apache.sentry.api.service.thrift.TSentryGrantOption;
 import org.apache.sentry.api.service.thrift.TSentryGroup;
 import org.apache.sentry.api.service.thrift.TSentryPrivilege;
 import org.apache.sentry.api.service.thrift.TSentryRole;
+import org.apache.sentry.provider.db.service.model.MSentryUser;
 import org.apache.sentry.provider.file.PolicyFile;
 import org.apache.sentry.api.common.SentryServiceUtil;
 import org.apache.sentry.service.common.ServiceConstants;
+import org.apache.sentry.service.common.ServiceConstants.SentryEntityType;
 import org.apache.sentry.service.common.ServiceConstants.ServerConfig;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -256,7 +258,7 @@ public class TestSentryStore extends org.junit.Assert {
     tSentryAuthorizable.setServer("server1");
 
     Set<TSentryPrivilege> privileges =
-        sentryStore.getTSentryPrivileges(new HashSet<String>(Arrays.asList(roleName)), tSentryAuthorizable);
+        sentryStore.getTSentryPrivileges(SentryEntityType.ROLE, new HashSet<String>(Arrays.asList(roleName)), tSentryAuthorizable);
 
     assertTrue(privileges.size() == 1);
 
@@ -581,12 +583,13 @@ public class TestSentryStore extends org.junit.Assert {
     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());
-    assertFalse(mPrivilege.getGrantOption());
+    for (MSentryPrivilege mPrivilege : privileges) {
+      assertEquals(server, mPrivilege.getServerName());
+      assertEquals(db, mPrivilege.getDbName());
+      assertEquals(table, mPrivilege.getTableName());
+      assertNotSame(AccessConstants.SELECT, mPrivilege.getAction());
+      assertFalse(mPrivilege.getGrantOption());
+    }
     long numDBPrivs = sentryStore.countMSentryPrivileges();
     assertEquals("Privilege count", numDBPrivs,1);
   }
@@ -676,9 +679,11 @@ public class TestSentryStore extends org.junit.Assert {
       // after having ALL and revoking SELECT, we should have INSERT
       MSentryRole role = sentryStore.getMSentryRoleByName(roleName);
       Set<MSentryPrivilege> privileges = role.getPrivileges();
-      assertEquals(privileges.toString(), i+1, privileges.size());
-      MSentryPrivilege mPrivilege = Iterables.get(privileges, 0);
-      assertEquals(AccessConstants.INSERT, mPrivilege.getAction());
+      assertEquals(privileges.toString(), 1 * (i+1), privileges.size());
+      for ( MSentryPrivilege mSentryPrivilege : privileges) {
+        assertNotSame(AccessConstants.INSERT, mSentryPrivilege.getAction());
+        assertNotSame(AccessConstants.ALL, mSentryPrivilege.getAction());
+      }
     }
 
     // Drop the role and clean up as before
@@ -726,7 +731,8 @@ public class TestSentryStore extends org.junit.Assert {
       priv.setGrantOption(TSentryGrantOption.UNSET);
       sentryStore.alterSentryRoleRevokePrivilege(grantor, roleName, priv);
 
-      sentryStore.findOrphanedPrivileges();
+      assertFalse(sentryStore.findOrphanedPrivileges());
+
       //After having ALL and revoking SELECT, we should have INSERT
       //Remove the INSERT privilege as well.
       //There should not be any more privileges in the sentry store
@@ -736,6 +742,23 @@ public class TestSentryStore extends org.junit.Assert {
       MSentryRole role = sentryStore.getMSentryRoleByName(roleName);
       assertEquals("Privilege Count", 0, role.getPrivileges().size());
 
+      priv.setAction(AccessConstants.CREATE);
+      sentryStore.alterSentryRoleRevokePrivilege(grantor, roleName, priv);
+
+      role = sentryStore.getMSentryRoleByName(roleName);
+      assertEquals("Privilege Count", 0, role.getPrivileges().size());
+
+      priv.setAction(AccessConstants.DROP);
+      sentryStore.alterSentryRoleRevokePrivilege(grantor, roleName, priv);
+
+      role = sentryStore.getMSentryRoleByName(roleName);
+      assertEquals("Privilege Count", 0, role.getPrivileges().size());
+
+      priv.setAction(AccessConstants.ALTER);
+      sentryStore.alterSentryRoleRevokePrivilege(grantor, roleName, priv);
+
+      role = sentryStore.getMSentryRoleByName(roleName);
+      assertEquals("Privilege Count", 0, role.getPrivileges().size());
     }
 
     // Drop the role and clean up as before
@@ -781,7 +804,7 @@ public class TestSentryStore extends org.junit.Assert {
     // Make sure we really have the expected number of privs in the database
     assertEquals(sentryStore.countMSentryPrivileges(), NUM_PRIVS);
 
-    //Revoking INSERT privilege. This is change the privilege to SELECT
+    //Revoking INSERT privilege. This is change the privilege to SELECT, CREATE, DROP, ALTER
     TSentryPrivilege priv = new TSentryPrivilege();
     priv.setPrivilegeScope("TABLE");
     priv.setServerName(server);
@@ -803,11 +826,13 @@ public class TestSentryStore extends org.junit.Assert {
     assertNotNull(mPriv);
 
     MSentryRole role = sentryStore.getMSentryRoleByName(roleName);
+
+    // should have NUM_PRIVS - 1 ALL privileges, and 4 privileges (SELECT, CREATE, DROP, ALTER)
     assertEquals("Privilege Count", NUM_PRIVS, role.getPrivileges().size());
 
     sentryStore.alterSentryRoleRevokePrivilege(grantor, roleName, priv);
     role = sentryStore.getMSentryRoleByName(roleName);
-    assertEquals("Privilege Count", NUM_PRIVS-1, role.getPrivileges().size());
+    assertEquals("Privilege Count", NUM_PRIVS - 1, role.getPrivileges().size());
 
   }
 
@@ -948,7 +973,8 @@ public class TestSentryStore extends org.junit.Assert {
     privilege.setAction(AccessConstants.SELECT);
     sentryStore.alterSentryRoleRevokePrivilege(grantor, roleName, privilege);
 
-    // At this point c1 has ALL privileges and c2 should have INSERT after revoking SELECT
+    // At this point c1 has ALL privileges and c2 should have (INSERT, CREATE, DROP, ALTER)
+    // after revoking SELECT
     role = sentryStore.getMSentryRoleByName(roleName);
     privileges = role.getPrivileges();
     assertEquals(privileges.toString(), 2, privileges.size());
@@ -960,14 +986,15 @@ public class TestSentryStore extends org.junit.Assert {
       if (mPrivilege.getColumnName().equals(column1)) {
         assertEquals(AccessConstants.ALL, mPrivilege.getAction());
       } else if (mPrivilege.getColumnName().equals(column2)) {
-        assertEquals(AccessConstants.INSERT, mPrivilege.getAction());
+        assertNotSame(AccessConstants.SELECT, mPrivilege.getAction());
+        assertNotSame(AccessConstants.ALL, mPrivilege.getAction());
       } else {
         fail("Unexpected column name: " + mPrivilege.getColumnName());
       }
     }
 
-    // after revoking INSERT table level privilege will remove privileges from column2
-    // and downgrade column1 to SELECT privileges.
+    // after revoking INSERT table level privilege will remove INSERT privileges from column2
+    // and downgrade column1 to (SELECT) privileges.
     privilege = new TSentryPrivilege();
     privilege.setPrivilegeScope("TABLE");
     privilege.setServerName(server);
@@ -979,8 +1006,6 @@ public class TestSentryStore extends org.junit.Assert {
     role = sentryStore.getMSentryRoleByName(roleName);
     privileges = role.getPrivileges();
     assertEquals(privileges.toString(), 1, privileges.size());
-    assertEquals(column1, Iterables.get(privileges, 0).getColumnName());
-    assertEquals(AccessConstants.SELECT, Iterables.get(privileges, 0).getAction());
 
     // Revoke ALL from the table should now remove all the column privileges.
     privilege.setAction(AccessConstants.ALL);
@@ -1023,15 +1048,11 @@ public class TestSentryStore extends org.junit.Assert {
     // Revoke SELECT on table2
     privilegeTable2.setAction(AccessConstants.SELECT);
     sentryStore.alterSentryRoleRevokePrivilege(grantor, roleName, privilegeTable2);
-    // after having ALL and revoking SELECT, we should have INSERT
+    // after having ALL and revoking SELECT, we should have (INSERT) at table2
     role = sentryStore.getMSentryRoleByName(roleName);
     privileges = role.getPrivileges();
     assertEquals(privileges.toString(), 2, privileges.size());
 
-    // At this point table1 has ALL privileges and table2 should have INSERT after revoking SELECT
-    role = sentryStore.getMSentryRoleByName(roleName);
-    privileges = role.getPrivileges();
-    assertEquals(privileges.toString(), 2, privileges.size());
     for (MSentryPrivilege mPrivilege: privileges) {
       assertEquals(server, mPrivilege.getServerName());
       assertEquals(db, mPrivilege.getDbName());
@@ -1039,7 +1060,8 @@ public class TestSentryStore extends org.junit.Assert {
       if (mPrivilege.getTableName().equals(table1)) {
         assertEquals(AccessConstants.ALL, mPrivilege.getAction());
       } else if (mPrivilege.getTableName().equals(table2)) {
-        assertEquals(AccessConstants.INSERT, mPrivilege.getAction());
+        assertNotSame(AccessConstants.SELECT, mPrivilege.getAction());
+        assertNotSame(AccessConstants.ALL, mPrivilege.getAction());
       } else {
         fail("Unexpected table name: " + mPrivilege.getTableName());
       }
@@ -1053,15 +1075,23 @@ public class TestSentryStore extends org.junit.Assert {
     role = sentryStore.getMSentryRoleByName(roleName);
     privileges = role.getPrivileges();
 
-    // after revoking INSERT database level privilege will remove privileges from table2
-    // and downgrade table1 to SELECT privileges.
+    // after revoking INSERT database level privilege
+    // table1 should have (SELECT)
+    // table2 should have ()
     assertEquals(privileges.toString(), 1, privileges.size());
-    MSentryPrivilege mPrivilege = Iterables.get(privileges, 0);
-    assertEquals(server, mPrivilege.getServerName());
-    assertEquals(db, mPrivilege.getDbName());
-    assertEquals(table1, mPrivilege.getTableName());
-    assertEquals(AccessConstants.SELECT, mPrivilege.getAction());
-    assertFalse(mPrivilege.getGrantOption());
+    for (MSentryPrivilege mPrivilege : privileges) {
+      assertEquals(server, mPrivilege.getServerName());
+      assertEquals(db, mPrivilege.getDbName());
+      if (table1.equals(mPrivilege.getTableName())) {
+        assertNotSame(AccessConstants.INSERT, mPrivilege.getAction());
+        assertNotSame(AccessConstants.ALL, mPrivilege.getAction());
+      } else if (table2.equals(mPrivilege.getTableName())) {
+        assertNotSame(AccessConstants.INSERT, mPrivilege.getAction());
+        assertNotSame(AccessConstants.SELECT, mPrivilege.getAction());
+        assertNotSame(AccessConstants.ALL, mPrivilege.getAction());
+      }
+      assertFalse(mPrivilege.getGrantOption());
+    }
   }
 
   /**
@@ -1098,15 +1128,11 @@ public class TestSentryStore extends org.junit.Assert {
     // Revoke SELECT on column2
     privilegeCol2.setAction(AccessConstants.SELECT);
     sentryStore.alterSentryRoleRevokePrivilege(grantor, roleName, privilegeCol2);
-    // after having ALL and revoking SELECT, we should have INSERT
+    // after having ALL and revoking SELECT, we should have (INSERT)
     role = sentryStore.getMSentryRoleByName(roleName);
     privileges = role.getPrivileges();
     assertEquals(privileges.toString(), 2, privileges.size());
 
-    // At this point column1 has ALL privileges and column2 should have INSERT after revoking SELECT
-    role = sentryStore.getMSentryRoleByName(roleName);
-    privileges = role.getPrivileges();
-    assertEquals(privileges.toString(), 2, privileges.size());
     for (MSentryPrivilege mPrivilege: privileges) {
       assertEquals(server, mPrivilege.getServerName());
       assertEquals(db, mPrivilege.getDbName());
@@ -1115,7 +1141,7 @@ public class TestSentryStore extends org.junit.Assert {
       if (mPrivilege.getColumnName().equals(column1)) {
         assertEquals(AccessConstants.ALL, mPrivilege.getAction());
       } else if (mPrivilege.getColumnName().equals(column2)) {
-        assertEquals(AccessConstants.INSERT, mPrivilege.getAction());
+        assertNotSame(AccessConstants.SELECT, mPrivilege.getAction());
       } else {
         fail("Unexpected column name: " + mPrivilege.getColumnName());
       }
@@ -1130,16 +1156,22 @@ public class TestSentryStore extends org.junit.Assert {
     role = sentryStore.getMSentryRoleByName(roleName);
     privileges = role.getPrivileges();
 
-    // after revoking INSERT database level privilege will remove privileges from column2
-    // and downgrade column1 to SELECT privileges.
+    // after revoking INSERT database level privilege
+    // column2 has ()
+    // and downgrade column1 to (SELECT) privileges.
     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(column1, mPrivilege.getColumnName());
-    assertEquals(AccessConstants.SELECT, mPrivilege.getAction());
-    assertFalse(mPrivilege.getGrantOption());
+    for (MSentryPrivilege mPrivilege : privileges) {
+      assertEquals(server, mPrivilege.getServerName());
+      assertEquals(db, mPrivilege.getDbName());
+      assertEquals(table, mPrivilege.getTableName());
+      if (column1.equals(mPrivilege.getColumnName())) {
+        assertNotSame(AccessConstants.INSERT, mPrivilege.getAction());
+      } else if (column1.equals(mPrivilege.getColumnName())) {
+        assertNotSame(AccessConstants.SELECT, mPrivilege.getAction());
+        assertNotSame(AccessConstants.INSERT, mPrivilege.getAction());
+      }
+      assertFalse(mPrivilege.getGrantOption());
+    }
   }
 
   @Test
@@ -1189,14 +1221,15 @@ public class TestSentryStore extends org.junit.Assert {
     privilege.setAction(AccessConstants.SELECT);
     privilege.setGrantOption(TSentryGrantOption.UNSET);
     sentryStore.alterSentryRoleRevokePrivilege(grantor, roleName, privilege);
-    // after having ALL and revoking SELECT, we should have INSERT
+    // 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(AccessConstants.INSERT, mPrivilege.getAction());
+    for (MSentryPrivilege mPrivilege : privileges) {
+      assertEquals(server, mPrivilege.getServerName());
+      assertEquals(db, mPrivilege.getDbName());
+      assertNotSame(AccessConstants.SELECT, mPrivilege.getAction());
+    }
   }
 
   @Test
@@ -2347,7 +2380,7 @@ public class TestSentryStore extends org.junit.Assert {
     tSentryAuthorizable.setServer("server1");
 
     Set<TSentryPrivilege> privileges =
-        sentryStore.getTSentryPrivileges(new HashSet<String>(Arrays.asList(roleName)), tSentryAuthorizable);
+        sentryStore.getTSentryPrivileges(SentryEntityType.ROLE, new HashSet<String>(Arrays.asList(roleName)), tSentryAuthorizable);
 
     assertTrue(privileges.size() == 1);
 
@@ -2388,7 +2421,7 @@ public class TestSentryStore extends org.junit.Assert {
     tSentryAuthorizable.setServer("server1");
 
     Set<TSentryPrivilege> privileges =
-        sentryStore.getTSentryPrivileges(new HashSet<String>(Arrays.asList(roleName)), tSentryAuthorizable);
+        sentryStore.getTSentryPrivileges(SentryEntityType.ROLE, new HashSet<String>(Arrays.asList(roleName)), tSentryAuthorizable);
 
     assertTrue(privileges.size() == 1);
 
@@ -3766,4 +3799,238 @@ public class TestSentryStore extends org.junit.Assert {
     nodeMap = pathDump.getNodeMap();
     assertEquals(7, nodeMap.size());
   }
+
+  /**
+   * Create a user with the given name and verify that it is created
+   *
+   * @param userName
+   * @throws Exception
+   */
+  private void createUser(String userName) throws Exception {
+    checkUserDoesNotExist(userName);
+    sentryStore.createSentryUser(userName);
+    checkUserExists(userName);
+  }
+
+  /**
+   * Fail test if user already exists
+   * @param userName User name to checl
+   * @throws Exception
+   */
+  private void checkUserDoesNotExist(String userName) throws Exception {
+    try {
+      sentryStore.getMSentryUserByName(userName);
+      fail("User " + userName + "already exists");
+    } catch (SentryNoSuchObjectException e) {
+      // Ok
+    }
+  }
+
+  /**
+   * Fail test if user doesn't exist
+   * @param userName User name to checl
+   * @throws Exception
+   */
+  private void checkUserExists(String userName) throws Exception {
+    assertEquals(userName.toLowerCase(),
+        sentryStore.getMSentryUserByName(userName).getUserName());
+  }
+
+  @Test
+  public void testGrantRevokePrivilegeForUser() throws Exception {
+    String userName = "test-privilege";
+    String grantor = "g1";
+    String server = "server1";
+    String db = "db1";
+    String table = "tbl1";
+    createUser(userName);
+    TSentryPrivilege privilege = new TSentryPrivilege();
+    privilege.setPrivilegeScope("TABLE");
+    privilege.setServerName(server);
+    privilege.setDbName(db);
+    privilege.setTableName(table);
+    privilege.setAction(AccessConstants.ALL);
+    privilege.setCreateTime(System.currentTimeMillis());
+    sentryStore.alterSentryUserGrantPrivilege(grantor, userName, privilege);
+    MSentryUser user = sentryStore.getMSentryUserByName(userName);
+    Set<MSentryPrivilege> privileges = user.getPrivileges();
+    assertEquals(privileges.toString(), 1, privileges.size());
+    privilege.setAction(AccessConstants.SELECT);
+    sentryStore.alterSentryUserRevokePrivilege(grantor, userName, privilege);
+    // after having ALL and revoking SELECT, we should have (INSERT)
+    user = sentryStore.getMSentryUserByName(userName);
+    privileges = user.getPrivileges();
+    assertEquals(privileges.toString(), 1, privileges.size());
+    for (MSentryPrivilege mPrivilege : privileges) {
+      assertEquals(server, mPrivilege.getServerName());
+      assertEquals(db, mPrivilege.getDbName());
+      assertEquals(table, mPrivilege.getTableName());
+      assertNotSame(AccessConstants.SELECT, mPrivilege.getAction());
+      assertFalse(mPrivilege.getGrantOption());
+    }
+    long numDBPrivs = sentryStore.countMSentryPrivileges();
+    assertEquals("Privilege count", numDBPrivs,1);
+
+    privilege.setAction(AccessConstants.INSERT);
+    sentryStore.alterSentryUserRevokePrivilege(grantor, userName, privilege);
+    user = sentryStore.getMSentryUserByName(userName);
+    privileges = user.getPrivileges();
+    assertEquals(privileges.toString(), 0, privileges.size());
+
+    sentryStore.dropSentryUser(userName);
+  }
+
+  /**
+   * Test after granting DB ALL privilege, can still grant table ALL privilege
+   * @throws Exception
+   */
+  @Test
+  public void testGrantDuplicatePrivilegeHierchy() throws Exception {
+    // grant database all privilege
+    String roleName = "test-privilege";
+    String grantor = "g1";
+    String server = "server1";
+    String db = "db1";
+    String table = "tbl1";
+    createRole(roleName);
+    TSentryPrivilege privilege = new TSentryPrivilege();
+    privilege.setPrivilegeScope("DATABASE");
+    privilege.setServerName(server);
+    privilege.setDbName(db);
+    privilege.setAction(AccessConstants.ALL);
+    privilege.setCreateTime(System.currentTimeMillis());
+    sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName, privilege);
+    MSentryRole role = sentryStore.getMSentryRoleByName(roleName);
+    Set<MSentryPrivilege> privileges = role.getPrivileges();
+    assertEquals(privileges.toString(), 1, privileges.size());
+
+    // grant table all privlege
+    privilege.setPrivilegeScope("TABLE");
+    privilege.setServerName(server.toUpperCase());
+    privilege.setDbName(db.toUpperCase());
+    privilege.setTableName(table.toUpperCase());
+    sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName, privilege);
+
+    // check if the table privilege is created
+    role = sentryStore.getMSentryRoleByName(roleName);
+    privileges = role.getPrivileges();
+    assertEquals(privileges.toString(), 2, privileges.size());
+  }
+
+  @Test
+  public void testListSentryPrivilegesForProviderForUser() throws Exception {
+    String userName1 = "list-privs-user1";
+    String userName2 = "list-privs-user2";
+    String grantor = "g1";
+    sentryStore.createSentryUser(userName1);
+    sentryStore.createSentryUser(userName2);
+
+    TSentryPrivilege privilege1 = new TSentryPrivilege();
+    privilege1.setPrivilegeScope("TABLE");
+    privilege1.setServerName("server1");
+    privilege1.setDbName("db1");
+    privilege1.setTableName("tbl1");
+    privilege1.setAction("SELECT");
+    privilege1.setCreateTime(System.currentTimeMillis());
+    sentryStore.alterSentryUserGrantPrivilege(grantor, userName1, privilege1);
+
+    privilege1.setAction("ALL");
+    sentryStore.alterSentryUserGrantPrivilege(grantor, userName2, privilege1);
+
+    assertEquals(Sets.newHashSet("server=server1->db=db1->table=tbl1->action=select"),
+        SentryStore.toTrimedLower(sentryStore.listAllSentryPrivilegesForProvider(
+            new HashSet<String>(),
+            Sets.newHashSet(userName1),
+            new TSentryActiveRoleSet(true, new HashSet<String>()))));
+  }
+
+  @Test
+  public void testGrantRevokePrivilegeMultipleTimesForRole() throws Exception {
+    String roleName = "test-privilege";
+    String grantor = "g1";
+    String server = "server1";
+    String db = "db1";
+    String table = "tbl1";
+    createRole(roleName);
+    TSentryPrivilege privilege = new TSentryPrivilege();
+    privilege.setPrivilegeScope("TABLE");
+    privilege.setServerName(server);
+    privilege.setDbName(db);
+    privilege.setTableName(table);
+    privilege.setAction(AccessConstants.ALL);
+    privilege.setCreateTime(System.currentTimeMillis());
+    sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName, privilege);
+
+    MSentryRole role = sentryStore.getMSentryRoleByName(roleName);
+    Set<MSentryPrivilege> privileges = role.getPrivileges();
+    assertEquals(privileges.toString(), 1, privileges.size());
+
+    privilege.setAction(AccessConstants.SELECT);
+    sentryStore.alterSentryRoleRevokePrivilege(grantor, roleName, privilege);
+    // after having ALL and revoking SELECT, we should have (INSERT)
+    role = sentryStore.getMSentryRoleByName(roleName);
+    privileges = role.getPrivileges();
+    assertEquals(privileges.toString(), 1, privileges.size());
+
+    // second round
+    privilege.setAction(AccessConstants.ALL);
+    sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName, privilege);
+    role = sentryStore.getMSentryRoleByName(roleName);
+    privileges = role.getPrivileges();
+    assertEquals(privileges.toString(), 1, privileges.size());
+
+    privilege.setAction(AccessConstants.INSERT);
+    sentryStore.alterSentryRoleRevokePrivilege(grantor, roleName, privilege);
+    // after having ALL and revoking INSERT, we should have (SELECT)
+    role = sentryStore.getMSentryRoleByName(roleName);
+    privileges = role.getPrivileges();
+    assertEquals(privileges.toString(), 1, privileges.size());
+
+    sentryStore.dropSentryRole(roleName);
+  }
+
+  @Test
+  public void testGrantRevokePrivilegeMultipleTimesForUser() throws Exception {
+    String userName = "test-privilege";
+    String grantor = "g1";
+    String server = "server1";
+    String db = "db1";
+    String table = "tbl1";
+    createUser(userName);
+    TSentryPrivilege privilege = new TSentryPrivilege();
+    privilege.setPrivilegeScope("TABLE");
+    privilege.setServerName(server);
+    privilege.setDbName(db);
+    privilege.setTableName(table);
+    privilege.setAction(AccessConstants.ALL);
+    privilege.setCreateTime(System.currentTimeMillis());
+    sentryStore.alterSentryUserGrantPrivilege(grantor, userName, privilege);
+
+    MSentryUser user = sentryStore.getMSentryUserByName(userName);
+    Set<MSentryPrivilege> privileges = user.getPrivileges();
+    assertEquals(privileges.toString(), 1, privileges.size());
+
+    privilege.setAction(AccessConstants.SELECT);
+    sentryStore.alterSentryUserRevokePrivilege(grantor, userName, privilege);
+    // after having ALL and revoking SELECT, we should have (INSERT)
+    user = sentryStore.getMSentryUserByName(userName);
+    privileges = user.getPrivileges();
+    assertEquals(privileges.toString(), 1, privileges.size());
+
+    // second round
+    privilege.setAction(AccessConstants.ALL);
+    sentryStore.alterSentryUserGrantPrivilege(grantor, userName, privilege);
+    user = sentryStore.getMSentryUserByName(userName);
+    privileges = user.getPrivileges();
+    assertEquals(privileges.toString(), 1, privileges.size());
+
+    privilege.setAction(AccessConstants.INSERT);
+    sentryStore.alterSentryUserRevokePrivilege(grantor, userName, privilege);
+    // after having ALL and revoking INSERT, we should have (SELECT)
+    user = sentryStore.getMSentryUserByName(userName);
+    privileges = user.getPrivileges();
+    assertEquals(privileges.toString(), 1, privileges.size());
+
+    sentryStore.dropSentryUser(userName);
+  }
 }