You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by ha...@apache.org on 2017/02/03 07:15:40 UTC

[1/2] sentry git commit: SENTRY-1536: Refactor SentryStore transaction management to allow for extra transactions for a single permission update (Hao Hao, Reviewed by: Alexander Kolbasov, Kalyan Kumar Kalvagadda and Lei Xu)

Repository: sentry
Updated Branches:
  refs/heads/sentry-ha-redesign b98b587e9 -> 2911c532b


http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java
index 1f05ba9..ecd1175 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/HMSFollower.java
@@ -32,8 +32,11 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hive.hcatalog.messaging.HCatEventMessage;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.core.common.exception.*;
+import org.apache.sentry.hdfs.PermissionsUpdate;
 import org.apache.sentry.hdfs.UpdateableAuthzPaths;
 import org.apache.sentry.hdfs.FullUpdateInitializer;
+import org.apache.sentry.hdfs.service.thrift.TPrivilegeChanges;
+import org.apache.sentry.provider.db.SentryPolicyStorePlugin;
 import org.apache.sentry.provider.db.service.persistent.SentryStore;
 import org.apache.sentry.provider.db.service.thrift.TSentryAuthorizable;
 import org.apache.thrift.TException;
@@ -51,6 +54,7 @@ import java.util.List;
 
 import static org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars.AUTHZ_SYNC_CREATE_WITH_POLICY_STORE;
 import static org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars.AUTHZ_SYNC_DROP_WITH_POLICY_STORE;
+import static org.apache.sentry.hdfs.Updateable.Update;
 
 /**
  * HMSFollower is the thread which follows the Hive MetaStore state changes from Sentry.
@@ -437,14 +441,16 @@ public class HMSFollower implements Runnable {
   private void dropSentryDbPrivileges(String dbName) throws Exception {
     TSentryAuthorizable authorizable = new TSentryAuthorizable(hiveInstance);
     authorizable.setDb(dbName);
-    sentryStore.dropPrivilege(authorizable);
+    sentryStore.dropPrivilege(authorizable, onDropSentryPrivilege(authorizable));
   }
+
   private void dropSentryTablePrivileges(String dbName, String tableName) throws Exception {
     TSentryAuthorizable authorizable = new TSentryAuthorizable(hiveInstance);
     authorizable.setDb(dbName);
     authorizable.setTable(tableName);
-    sentryStore.dropPrivilege(authorizable);
+    sentryStore.dropPrivilege(authorizable, onDropSentryPrivilege(authorizable));
   }
+
   private void renamePrivileges(String oldDbName, String oldTableName, String newDbName, String newTableName) throws
       Exception {
     TSentryAuthorizable oldAuthorizable = new TSentryAuthorizable(hiveInstance);
@@ -453,6 +459,43 @@ public class HMSFollower implements Runnable {
     TSentryAuthorizable newAuthorizable = new TSentryAuthorizable(hiveInstance);
     newAuthorizable.setDb(newDbName);
     newAuthorizable.setTable(newTableName);
-    sentryStore.renamePrivilege(oldAuthorizable, newAuthorizable);
+    Update update =
+        onRenameSentryPrivilege(oldAuthorizable, newAuthorizable);
+    sentryStore.renamePrivilege(oldAuthorizable, newAuthorizable, update);
+  }
+
+  @VisibleForTesting
+  static Update onDropSentryPrivilege(TSentryAuthorizable authorizable) {
+    PermissionsUpdate update = new PermissionsUpdate(SentryStore.INIT_CHANGE_ID, false);
+    String authzObj = getAuthzObj(authorizable);
+    update.addPrivilegeUpdate(authzObj).putToDelPrivileges(PermissionsUpdate.ALL_ROLES, PermissionsUpdate.ALL_ROLES);
+    return update;
+  }
+
+  @VisibleForTesting
+  static Update onRenameSentryPrivilege(TSentryAuthorizable oldAuthorizable,
+            TSentryAuthorizable newAuthorizable)
+          throws SentryPolicyStorePlugin.SentryPluginException {
+    String oldAuthz = getAuthzObj(oldAuthorizable);
+    String newAuthz = getAuthzObj(newAuthorizable);
+    PermissionsUpdate update = new PermissionsUpdate(SentryStore.INIT_CHANGE_ID, false);
+    TPrivilegeChanges privUpdate = update.addPrivilegeUpdate(PermissionsUpdate.RENAME_PRIVS);
+    privUpdate.putToAddPrivileges(newAuthz, newAuthz);
+    privUpdate.putToDelPrivileges(oldAuthz, oldAuthz);
+    return update;
+  }
+
+  public static String getAuthzObj(TSentryAuthorizable authzble) {
+    String authzObj = null;
+    if (!SentryStore.isNULL(authzble.getDb())) {
+      String dbName = authzble.getDb();
+      String tblName = authzble.getTable();
+      if (SentryStore.isNULL(tblName)) {
+        authzObj = dbName;
+      } else {
+        authzObj = dbName + "." + tblName;
+      }
+    }
+    return authzObj == null ? null : authzObj.toLowerCase();
   }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/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 a35c8d7..dfaac15 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
@@ -35,6 +35,10 @@ import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.core.common.exception.SentryAlreadyExistsException;
 import org.apache.sentry.core.common.exception.SentryGrantDeniedException;
 import org.apache.sentry.core.common.exception.SentryNoSuchObjectException;
+import org.apache.sentry.hdfs.PermissionsUpdate;
+import org.apache.sentry.hdfs.service.thrift.TPrivilegeChanges;
+import org.apache.sentry.hdfs.service.thrift.TRoleChanges;
+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.MSentryRole;
 import org.apache.sentry.provider.db.service.thrift.TSentryActiveRoleSet;
@@ -57,6 +61,8 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.io.Files;
 
+import static org.apache.sentry.hdfs.Updateable.Update;
+
 public class TestSentryStore extends org.junit.Assert {
 
   private static File dataDir;
@@ -2263,6 +2269,208 @@ public class TestSentryStore extends org.junit.Assert {
     assertTrue(names.containsAll(result));
   }
 
+  @Test
+  public void testPrivilegesWithPermUpdate() throws Exception {
+    String roleName = "test-privilege";
+    String grantor = "g1";
+    String server = "server1";
+    String db = "db1";
+    String table = "tbl1";
+    String authzObj = "db1.tbl1";
+    createRole(roleName);
+
+    TSentryPrivilege privilege = new TSentryPrivilege();
+    privilege.setPrivilegeScope("Column");
+    privilege.setServerName(server);
+    privilege.setDbName(db);
+    privilege.setTableName(table);
+    privilege.setAction(AccessConstants.SELECT);
+    privilege.setCreateTime(System.currentTimeMillis());
+
+    // Generate the permission add update authzObj "db1.tbl1"
+    PermissionsUpdate addUpdate = new PermissionsUpdate(0, false);
+    addUpdate.addPrivilegeUpdate(authzObj).putToAddPrivileges(
+        roleName, privilege.getAction().toUpperCase());
+
+    // Grant the privilege to role test-privilege and verify it has been persisted.
+    Map<TSentryPrivilege, Update> addPrivilegesUpdateMap = Maps.newHashMap();
+    addPrivilegesUpdateMap.put(privilege, addUpdate);
+    sentryStore.alterSentryRoleGrantPrivileges(grantor, roleName, Sets.newHashSet(privilege), addPrivilegesUpdateMap);
+    MSentryRole role = sentryStore.getMSentryRoleByName(roleName);
+    Set<MSentryPrivilege> privileges = role.getPrivileges();
+    assertEquals(privileges.toString(), 1, privileges.size());
+
+    // Query the persisted perm change and ensure it equals to the original one
+    long lastChangeID = sentryStore.getLastProcessedPermChangeID();
+    MSentryPermChange addPermChange = sentryStore.getMSentryPermChangeByID(lastChangeID);
+    assertEquals(addUpdate.JSONSerialize(), addPermChange.getPermChange());
+
+    // Generate the permission delete update authzObj "db1.tbl1"
+    PermissionsUpdate delUpdate = new PermissionsUpdate(0, false);
+    delUpdate.addPrivilegeUpdate(authzObj).putToDelPrivileges(
+        roleName, privilege.getAction().toUpperCase());
+
+    // Revoke the same privilege and verify it has been removed.
+    Map<TSentryPrivilege, Update> delPrivilegesUpdateMap = Maps.newHashMap();
+    delPrivilegesUpdateMap.put(privilege, delUpdate);
+    sentryStore.alterSentryRoleRevokePrivileges(grantor, roleName,
+        Sets.newHashSet(privilege), delPrivilegesUpdateMap);
+    role = sentryStore.getMSentryRoleByName(roleName);
+    privileges = role.getPrivileges();
+    assertEquals(0, privileges.size());
+
+    // Query the persisted perm change and ensure it equals to the original one
+    MSentryPermChange delPermChange = sentryStore.getMSentryPermChangeByID(lastChangeID + 1);
+    assertEquals(delUpdate.JSONSerialize(), delPermChange.getPermChange());
+  }
+
+  @Test
+  public void testAddDeleteGroupsWithPermUpdate() throws Exception {
+    String roleName = "test-groups";
+    String grantor = "g1";
+    createRole(roleName);
+
+    Set<TSentryGroup> groups = Sets.newHashSet();
+    TSentryGroup group = new TSentryGroup();
+    group.setGroupName("test-groups-g1");
+    groups.add(group);
+    group = new TSentryGroup();
+    group.setGroupName("test-groups-g2");
+    groups.add(group);
+
+    // Generate the permission add update for role "test-groups"
+    PermissionsUpdate addUpdate = new PermissionsUpdate(0, false);
+    TRoleChanges addrUpdate = addUpdate.addRoleUpdate(roleName);
+    for (TSentryGroup g : groups) {
+      addrUpdate.addToAddGroups(g.getGroupName());
+    }
+
+    // Assign the role "test-groups" to the groups and verify.
+    sentryStore.alterSentryRoleAddGroups(grantor, roleName, groups, addUpdate);
+    MSentryRole role = sentryStore.getMSentryRoleByName(roleName);
+    assertEquals(2, role.getGroups().size());
+
+    // Query the persisted perm change and ensure it equals to the original one
+    long lastChangeID = sentryStore.getLastProcessedPermChangeID();
+    MSentryPermChange addPermChange = sentryStore.getMSentryPermChangeByID(lastChangeID);
+    assertEquals(addUpdate.JSONSerialize(), addPermChange.getPermChange());
+
+    // Generate the permission add update for role "test-groups"
+    PermissionsUpdate delUpdate = new PermissionsUpdate(0, false);
+    TRoleChanges delrUpdate = delUpdate.addRoleUpdate(roleName);
+    for (TSentryGroup g : groups) {
+      delrUpdate.addToDelGroups(g.getGroupName());
+    }
+
+    // Revoke the role "test-groups" to the groups and verify.
+    sentryStore.alterSentryRoleDeleteGroups(roleName, groups, delUpdate);
+    role = sentryStore.getMSentryRoleByName(roleName);
+    assertEquals(Collections.emptySet(), role.getGroups());
+
+    // Query the persisted perm change and ensure it equals to the original one
+    MSentryPermChange delPermChange = sentryStore.getMSentryPermChangeByID(lastChangeID + 1);
+    assertEquals(delUpdate.JSONSerialize(), delPermChange.getPermChange());
+  }
+
+  @Test
+  public void testCreateDropRoleWithPermUpdate() throws Exception {
+    String roleName = "test-drop-role";
+    createRole(roleName);
+
+    // Generate the permission del update for dropping role "test-drop-role"
+    PermissionsUpdate delUpdate = new PermissionsUpdate(0, false);
+    delUpdate.addPrivilegeUpdate(PermissionsUpdate.ALL_AUTHZ_OBJ).putToDelPrivileges(
+            roleName, PermissionsUpdate.ALL_AUTHZ_OBJ);
+    delUpdate.addRoleUpdate(roleName).addToDelGroups(PermissionsUpdate.ALL_GROUPS);
+
+    // Drop the role and verify.
+    sentryStore.dropSentryRole(roleName, delUpdate);
+    checkRoleDoesNotExist(roleName);
+
+    // Query the persisted perm change and ensure it equals to the original one
+    long lastChangeID = sentryStore.getLastProcessedPermChangeID();
+    MSentryPermChange delPermChange = sentryStore.getMSentryPermChangeByID(lastChangeID);
+    assertEquals(delUpdate.JSONSerialize(), delPermChange.getPermChange());
+  }
+
+  @Test
+  public void testDropObjWithPermUpdate() throws Exception {
+    String roleName1 = "list-privs-r1", roleName2 = "list-privs-r2";
+    String grantor = "g1";
+    sentryStore.createSentryRole(roleName1);
+    sentryStore.createSentryRole(roleName2);
+
+    String authzObj = "db1.tbl1";
+    TSentryPrivilege privilege_tbl1 = new TSentryPrivilege();
+    privilege_tbl1.setPrivilegeScope("TABLE");
+    privilege_tbl1.setServerName("server1");
+    privilege_tbl1.setDbName("db1");
+    privilege_tbl1.setTableName("tbl1");
+    privilege_tbl1.setCreateTime(System.currentTimeMillis());
+    privilege_tbl1.setAction("SELECT");
+
+    sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName1, privilege_tbl1);
+
+    // Generate the permission drop update for dropping privilege for "db1.tbl1"
+    PermissionsUpdate dropUpdate = new PermissionsUpdate(0, false);
+    dropUpdate.addPrivilegeUpdate(authzObj).putToDelPrivileges(PermissionsUpdate.ALL_ROLES,
+            PermissionsUpdate.ALL_ROLES);
+
+    // Drop the privilege and verify.
+    sentryStore.dropPrivilege(toTSentryAuthorizable(privilege_tbl1), dropUpdate);
+    assertEquals(0, sentryStore.getAllTSentryPrivilegesByRoleName(roleName1).size());
+    assertEquals(0, sentryStore.getAllTSentryPrivilegesByRoleName(roleName2).size());
+
+    // Query the persisted perm change and ensure it equals to the original one
+    long lastChangeID = sentryStore.getLastProcessedPermChangeID();
+    MSentryPermChange dropPermChange = sentryStore.getMSentryPermChangeByID(lastChangeID);
+    assertEquals(dropUpdate.JSONSerialize(), dropPermChange.getPermChange());
+  }
+
+  @Test
+  public void testRenameObjWithPermUpdate() throws Exception {
+    String roleName1 = "role1", roleName2 = "role2", roleName3 = "role3";
+    String grantor = "g1";
+    String table1 = "tbl1", table2 = "tbl2";
+
+    sentryStore.createSentryRole(roleName1);
+
+    TSentryPrivilege privilege_tbl1 = new TSentryPrivilege();
+    privilege_tbl1.setPrivilegeScope("TABLE");
+    privilege_tbl1.setServerName("server1");
+    privilege_tbl1.setDbName("db1");
+    privilege_tbl1.setTableName(table1);
+    privilege_tbl1.setCreateTime(System.currentTimeMillis());
+    privilege_tbl1.setAction(AccessConstants.ALL);
+
+    sentryStore.alterSentryRoleGrantPrivilege(grantor, roleName1, privilege_tbl1);
+
+    // Generate the permission rename update for renaming privilege for "db1.tbl1"
+    String oldAuthz = "db1.tbl1";
+    String newAuthz = "db1.tbl2";
+    PermissionsUpdate renameUpdate = new PermissionsUpdate(0, false);
+    TPrivilegeChanges privUpdate = renameUpdate.addPrivilegeUpdate(PermissionsUpdate.RENAME_PRIVS);
+    privUpdate.putToAddPrivileges(newAuthz, newAuthz);
+    privUpdate.putToDelPrivileges(oldAuthz, oldAuthz);
+
+    // Rename the privilege and verify.
+    TSentryAuthorizable oldTable = toTSentryAuthorizable(privilege_tbl1);
+    TSentryAuthorizable newTable = toTSentryAuthorizable(privilege_tbl1);
+    newTable.setTable(table2);
+    sentryStore.renamePrivilege(oldTable, newTable, renameUpdate);
+
+    Set<TSentryPrivilege> privilegeSet = sentryStore.getAllTSentryPrivilegesByRoleName(roleName1);
+    assertEquals(1, privilegeSet.size());
+    for (TSentryPrivilege privilege : privilegeSet) {
+      assertTrue(table2.equalsIgnoreCase(privilege.getTableName()));
+    }
+
+    // Query the persisted perm change and ensure it equals to the original one
+    long lastChangeID = sentryStore.getLastProcessedPermChangeID();
+    MSentryPermChange renamePermChange = sentryStore.getMSentryPermChangeByID(lastChangeID);
+    assertEquals(renameUpdate.JSONSerialize(), renamePermChange.getPermChange());
+  }
+
   protected static void addGroupsToUser(String user, String... groupNames) {
     policyFile.addGroupsToUser(user, groupNames);
   }

http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestHMSFollower.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestHMSFollower.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestHMSFollower.java
index d601b1e..fd97936 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestHMSFollower.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestHMSFollower.java
@@ -17,13 +17,11 @@
 package org.apache.sentry.service.thrift;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.*;
 import org.apache.hive.hcatalog.messaging.HCatEventMessage;
 import org.apache.sentry.binding.metastore.messaging.json.SentryJSONMessageFactory;
 import org.apache.sentry.provider.db.service.persistent.SentryStore;
 import org.apache.sentry.provider.db.service.thrift.TSentryAuthorizable;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 
@@ -56,8 +54,9 @@ public class TestHMSFollower {
     authorizable.setServer(hiveInstance);
     authorizable.setDb("db1");
 
-    verify(sentryStore, times(1)).dropPrivilege(authorizable);
+    verify(sentryStore, times(1)).dropPrivilege(authorizable, HMSFollower.onDropSentryPrivilege(authorizable));
   }
+
   @Test
   public void testDropDatabase() throws Exception {
     String dbName = "db1";
@@ -76,7 +75,7 @@ public class TestHMSFollower {
     authorizable.setServer(hiveInstance);
     authorizable.setDb("db1");
 
-    verify(sentryStore, times(1)).dropPrivilege(authorizable);
+    verify(sentryStore, times(1)).dropPrivilege(authorizable, HMSFollower.onDropSentryPrivilege(authorizable)) ;
   }
   @Test
   public void testCreateTable() throws Exception {
@@ -100,7 +99,7 @@ public class TestHMSFollower {
     authorizable.setDb("db1");
     authorizable.setTable(tableName);
 
-    verify(sentryStore, times(1)).dropPrivilege(authorizable);
+    verify(sentryStore, times(1)).dropPrivilege(authorizable, HMSFollower.onDropSentryPrivilege(authorizable));
   }
   @Test
   public void testDropTable() throws Exception {
@@ -124,7 +123,7 @@ public class TestHMSFollower {
     authorizable.setDb("db1");
     authorizable.setTable(tableName);
 
-    verify(sentryStore, times(1)).dropPrivilege(authorizable);
+    verify(sentryStore, times(1)).dropPrivilege(authorizable, HMSFollower.onDropSentryPrivilege(authorizable));
   }
   @Test
   public void testRenameTable() throws Exception {
@@ -160,6 +159,6 @@ public class TestHMSFollower {
     newAuthorizable.setDb(newDbName);
     newAuthorizable.setTable(newTableName);
 
-    verify(sentryStore, times(1)).renamePrivilege(authorizable, newAuthorizable);
+    verify(sentryStore, times(1)).renamePrivilege(authorizable, newAuthorizable, HMSFollower.onRenameSentryPrivilege(authorizable, newAuthorizable));
   }
 }


[2/2] sentry git commit: SENTRY-1536: Refactor SentryStore transaction management to allow for extra transactions for a single permission update (Hao Hao, Reviewed by: Alexander Kolbasov, Kalyan Kumar Kalvagadda and Lei Xu)

Posted by ha...@apache.org.
SENTRY-1536: Refactor SentryStore transaction management to allow for extra transactions for a single permission update (Hao Hao, Reviewed by: Alexander Kolbasov, Kalyan Kumar Kalvagadda and Lei Xu)

Change-Id: I655363d7a65f5eb6a3f662793e719c8c482c64a0


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

Branch: refs/heads/sentry-ha-redesign
Commit: 2911c532bb83e75f60902fa75bb2097688881947
Parents: b98b587
Author: hahao <ha...@cloudera.com>
Authored: Thu Feb 2 21:28:44 2017 -0800
Committer: hahao <ha...@cloudera.com>
Committed: Thu Feb 2 23:13:51 2017 -0800

----------------------------------------------------------------------
 .../org/apache/sentry/hdfs/PathsUpdate.java     |  26 +
 .../apache/sentry/hdfs/PermissionsUpdate.java   |  27 +
 .../sentry/hdfs/UpdateableAuthzPermissions.java |   1 +
 .../org/apache/sentry/hdfs/SentryPlugin.java    | 106 ++--
 .../provider/db/SentryPolicyStorePlugin.java    |  30 +-
 .../service/persistent/DelegateSentryStore.java |  59 +-
 .../db/service/model/MSentryPathChange.java     |  35 +-
 .../db/service/model/MSentryPermChange.java     |  33 +-
 .../provider/db/service/model/package.jdo       |   4 +-
 .../persistent/DeltaTransactionBlock.java       |  94 +++
 .../db/service/persistent/SentryStore.java      | 574 +++++++++++++++++--
 .../service/persistent/TransactionManager.java  |  95 ++-
 .../thrift/SentryPolicyStoreProcessor.java      | 143 +++--
 .../sentry/service/thrift/HMSFollower.java      |  49 +-
 .../db/service/persistent/TestSentryStore.java  | 208 +++++++
 .../sentry/service/thrift/TestHMSFollower.java  |  13 +-
 16 files changed, 1254 insertions(+), 243 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PathsUpdate.java
----------------------------------------------------------------------
diff --git a/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PathsUpdate.java b/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PathsUpdate.java
index 9ecd9e4..ffb0756 100644
--- a/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PathsUpdate.java
+++ b/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PathsUpdate.java
@@ -173,4 +173,30 @@ public class PathsUpdate implements Updateable.Update {
     return ThriftSerializer.serializeToJSON(tPathsUpdate);
   }
 
+  @Override
+  public int hashCode() {
+    return (tPathsUpdate == null) ? 0 : tPathsUpdate.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) {
+      return false;
+    }
+
+    if (this == obj) {
+      return true;
+    }
+
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+
+    PathsUpdate other = (PathsUpdate) obj;
+    if (tPathsUpdate == null) {
+      return other.tPathsUpdate == null;
+    }
+    return tPathsUpdate.equals(other.tPathsUpdate);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PermissionsUpdate.java
----------------------------------------------------------------------
diff --git a/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PermissionsUpdate.java b/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PermissionsUpdate.java
index a346587..14a4a0f 100644
--- a/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PermissionsUpdate.java
+++ b/sentry-hdfs/sentry-hdfs-common/src/main/java/org/apache/sentry/hdfs/PermissionsUpdate.java
@@ -117,4 +117,31 @@ public class PermissionsUpdate implements Updateable.Update {
   public String JSONSerialize() throws TException {
     return ThriftSerializer.serializeToJSON(tPermUpdate);
   }
+
+  @Override
+  public int hashCode() {
+    return (tPermUpdate == null) ? 0 : tPermUpdate.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) {
+      return false;
+    }
+
+    if (this == obj) {
+      return true;
+    }
+
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+
+    PermissionsUpdate other = (PermissionsUpdate) obj;
+    if (tPermUpdate == null) {
+      return other.tPermUpdate == null;
+    }
+    return tPermUpdate.equals(other.tPermUpdate);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/sentry-hdfs/sentry-hdfs-namenode-plugin/src/main/java/org/apache/sentry/hdfs/UpdateableAuthzPermissions.java
----------------------------------------------------------------------
diff --git a/sentry-hdfs/sentry-hdfs-namenode-plugin/src/main/java/org/apache/sentry/hdfs/UpdateableAuthzPermissions.java b/sentry-hdfs/sentry-hdfs-namenode-plugin/src/main/java/org/apache/sentry/hdfs/UpdateableAuthzPermissions.java
index 2472928..766611c 100644
--- a/sentry-hdfs/sentry-hdfs-namenode-plugin/src/main/java/org/apache/sentry/hdfs/UpdateableAuthzPermissions.java
+++ b/sentry-hdfs/sentry-hdfs-namenode-plugin/src/main/java/org/apache/sentry/hdfs/UpdateableAuthzPermissions.java
@@ -217,6 +217,7 @@ public class UpdateableAuthzPermissions implements AuthzPermissions, Updateable<
 
   @Override
   public PermissionsUpdate createFullImageUpdate(long currSeqNum) {
+    // Using in-memory cache perms to create a full permission snapshot.
     PermissionsUpdate retVal = new PermissionsUpdate(currSeqNum, true);
     for (PrivilegeInfo pInfo : perms.getAllPrivileges()) {
       TPrivilegeChanges pUpdate = retVal.addPrivilegeUpdate(pInfo.getAuthzObj());

http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/sentry-hdfs/sentry-hdfs-service/src/main/java/org/apache/sentry/hdfs/SentryPlugin.java
----------------------------------------------------------------------
diff --git a/sentry-hdfs/sentry-hdfs-service/src/main/java/org/apache/sentry/hdfs/SentryPlugin.java b/sentry-hdfs/sentry-hdfs-service/src/main/java/org/apache/sentry/hdfs/SentryPlugin.java
index 47c9f9d..f68f690 100644
--- a/sentry-hdfs/sentry-hdfs-service/src/main/java/org/apache/sentry/hdfs/SentryPlugin.java
+++ b/sentry-hdfs/sentry-hdfs-service/src/main/java/org/apache/sentry/hdfs/SentryPlugin.java
@@ -34,6 +34,7 @@ import org.apache.sentry.hdfs.service.thrift.TPermissionsUpdate;
 import org.apache.sentry.hdfs.service.thrift.TPrivilegeChanges;
 import org.apache.sentry.hdfs.service.thrift.TRoleChanges;
 import org.apache.sentry.provider.db.SentryPolicyStorePlugin;
+import org.apache.sentry.provider.db.service.persistent.DeltaTransactionBlock;
 import org.apache.sentry.provider.db.service.persistent.SentryStore;
 import org.apache.sentry.provider.db.service.thrift.TAlterSentryRoleAddGroupsRequest;
 import org.apache.sentry.provider.db.service.thrift.TAlterSentryRoleDeleteGroupsRequest;
@@ -45,9 +46,12 @@ import org.apache.sentry.provider.db.service.thrift.TRenamePrivilegesRequest;
 import org.apache.sentry.provider.db.service.thrift.TSentryAuthorizable;
 import org.apache.sentry.provider.db.service.thrift.TSentryGroup;
 import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
+import org.apache.sentry.service.thrift.HMSFollower;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.sentry.hdfs.Updateable.Update;
+
   /**
    * SentryPlugin facilitates HDFS synchronization between HMS and NameNode.
    * <p>
@@ -162,12 +166,13 @@ public class SentryPlugin implements SentryPolicyStorePlugin, SigUtils.SigListen
 
   private UpdateForwarder<PathsUpdate> pathsUpdater;
   private UpdateForwarder<PermissionsUpdate> permsUpdater;
+  // TODO: Each perm change sequence number should be generated during persistence at sentry store.
   private final AtomicLong permSeqNum = new AtomicLong(5);
   private PermImageRetriever permImageRetriever;
   private boolean outOfSync = false;
   /*
    * This number is smaller than starting sequence numbers used by NN and HMS
-   * so in both cases its effect is to creat appearence of out-of-sync
+   * so in both cases its effect is to create appearance of out-of-sync
    * updates on the Sentry server (as if there were no previous updates at all).
    * It, in turn, triggers a) pushing full update from HMS to Sentry and
    * b) pulling full update from Sentry to NameNode.
@@ -287,19 +292,21 @@ public class SentryPlugin implements SentryPolicyStorePlugin, SigUtils.SigListen
   }
 
   @Override
-  public void onAlterSentryRoleAddGroups(
+  public Update onAlterSentryRoleAddGroups(
       TAlterSentryRoleAddGroupsRequest request) throws SentryPluginException {
     PermissionsUpdate update = new PermissionsUpdate(permSeqNum.incrementAndGet(), false);
     TRoleChanges rUpdate = update.addRoleUpdate(request.getRoleName());
     for (TSentryGroup group : request.getGroups()) {
       rUpdate.addToAddGroups(group.getGroupName());
     }
+
     permsUpdater.handleUpdateNotification(update);
     LOGGER.debug("Authz Perm preUpdate [" + update.getSeqNum() + ", " + request.getRoleName() + "]..");
+    return update;
   }
 
   @Override
-  public void onAlterSentryRoleDeleteGroups(
+  public Update onAlterSentryRoleDeleteGroups(
       TAlterSentryRoleDeleteGroupsRequest request)
           throws SentryPluginException {
     PermissionsUpdate update = new PermissionsUpdate(permSeqNum.incrementAndGet(), false);
@@ -307,58 +314,75 @@ public class SentryPlugin implements SentryPolicyStorePlugin, SigUtils.SigListen
     for (TSentryGroup group : request.getGroups()) {
       rUpdate.addToDelGroups(group.getGroupName());
     }
+
     permsUpdater.handleUpdateNotification(update);
     LOGGER.debug("Authz Perm preUpdate [" + update.getSeqNum() + ", " + request.getRoleName() + "]..");
+    return update;
   }
 
   @Override
-  public void onAlterSentryRoleGrantPrivilege(
-      TAlterSentryRoleGrantPrivilegeRequest request)
-          throws SentryPluginException {
+  public void onAlterSentryRoleGrantPrivilege(TAlterSentryRoleGrantPrivilegeRequest request,
+          Map<TSentryPrivilege, Update> privilegesUpdateMap) throws SentryPluginException {
+
     if (request.isSetPrivileges()) {
       String roleName = request.getRoleName();
+
       for (TSentryPrivilege privilege : request.getPrivileges()) {
         if(!("COLUMN".equalsIgnoreCase(privilege.getPrivilegeScope()))) {
-          onAlterSentryRoleGrantPrivilegeCore(roleName, privilege);
+          PermissionsUpdate update = onAlterSentryRoleGrantPrivilegeCore(roleName, privilege);
+          if (update != null && privilegesUpdateMap != null) {
+            privilegesUpdateMap.put(privilege, update);
+          }
         }
       }
     }
   }
 
-  private void onAlterSentryRoleGrantPrivilegeCore(String roleName, TSentryPrivilege privilege)
+  private PermissionsUpdate onAlterSentryRoleGrantPrivilegeCore(String roleName, TSentryPrivilege privilege)
       throws SentryPluginException {
     String authzObj = getAuthzObj(privilege);
-    if (authzObj != null) {
-      PermissionsUpdate update = new PermissionsUpdate(permSeqNum.incrementAndGet(), false);
-      update.addPrivilegeUpdate(authzObj).putToAddPrivileges(
-          roleName, privilege.getAction().toUpperCase());
-      permsUpdater.handleUpdateNotification(update);
-      LOGGER.debug("Authz Perm preUpdate [" + update.getSeqNum() + "]..");
+    if (authzObj == null) {
+      return null;
     }
+
+    PermissionsUpdate update = new PermissionsUpdate(permSeqNum.incrementAndGet(), false);
+    update.addPrivilegeUpdate(authzObj).putToAddPrivileges(
+        roleName, privilege.getAction().toUpperCase());
+
+    permsUpdater.handleUpdateNotification(update);
+    LOGGER.debug("Authz Perm preUpdate [" + update.getSeqNum() + "]..");
+    return update;
   }
 
   @Override
-  public void onRenameSentryPrivilege(TRenamePrivilegesRequest request)
+  public Update onRenameSentryPrivilege(TRenamePrivilegesRequest request)
       throws SentryPluginException {
-    String oldAuthz = getAuthzObj(request.getOldAuthorizable());
-    String newAuthz = getAuthzObj(request.getNewAuthorizable());
+    String oldAuthz = HMSFollower.getAuthzObj(request.getOldAuthorizable());
+    String newAuthz = HMSFollower.getAuthzObj(request.getNewAuthorizable());
     PermissionsUpdate update = new PermissionsUpdate(permSeqNum.incrementAndGet(), false);
     TPrivilegeChanges privUpdate = update.addPrivilegeUpdate(PermissionsUpdate.RENAME_PRIVS);
     privUpdate.putToAddPrivileges(newAuthz, newAuthz);
     privUpdate.putToDelPrivileges(oldAuthz, oldAuthz);
+
     permsUpdater.handleUpdateNotification(update);
     LOGGER.debug("Authz Perm preUpdate [" + update.getSeqNum() + ", " + newAuthz + ", " + oldAuthz + "]..");
+    return update;
   }
 
   @Override
-  public void onAlterSentryRoleRevokePrivilege(
-      TAlterSentryRoleRevokePrivilegeRequest request)
+  public void onAlterSentryRoleRevokePrivilege(TAlterSentryRoleRevokePrivilegeRequest request,
+      Map<TSentryPrivilege, Update> privilegesUpdateMap)
           throws SentryPluginException {
+
     if (request.isSetPrivileges()) {
       String roleName = request.getRoleName();
+
       for (TSentryPrivilege privilege : request.getPrivileges()) {
         if(!("COLUMN".equalsIgnoreCase(privilege.getPrivilegeScope()))) {
-          onAlterSentryRoleRevokePrivilegeCore(roleName, privilege);
+          PermissionsUpdate update = onAlterSentryRoleRevokePrivilegeCore(roleName, privilege);
+          if (update != null && privilegesUpdateMap != null) {
+            privilegesUpdateMap.put(privilege, update);
+          }
         }
       }
     }
@@ -372,38 +396,46 @@ public class SentryPlugin implements SentryPolicyStorePlugin, SigUtils.SigListen
     this.outOfSync = outOfSync;
   }
 
-  private void onAlterSentryRoleRevokePrivilegeCore(String roleName, TSentryPrivilege privilege)
+  private PermissionsUpdate onAlterSentryRoleRevokePrivilegeCore(String roleName, TSentryPrivilege privilege)
       throws SentryPluginException {
     String authzObj = getAuthzObj(privilege);
-    if (authzObj != null) {
-      PermissionsUpdate update = new PermissionsUpdate(permSeqNum.incrementAndGet(), false);
-      update.addPrivilegeUpdate(authzObj).putToDelPrivileges(
-          roleName, privilege.getAction().toUpperCase());
-      permsUpdater.handleUpdateNotification(update);
-      LOGGER.debug("Authz Perm preUpdate [" + update.getSeqNum() + ", " + authzObj + "]..");
+    if (authzObj == null) {
+      return null;
     }
+
+    PermissionsUpdate update = new PermissionsUpdate(permSeqNum.incrementAndGet(), false);
+    update.addPrivilegeUpdate(authzObj).putToDelPrivileges(
+        roleName, privilege.getAction().toUpperCase());
+
+    permsUpdater.handleUpdateNotification(update);
+    LOGGER.debug("Authz Perm preUpdate [" + update.getSeqNum() + ", " + authzObj + "]..");
+    return update;
   }
 
   @Override
-  public void onDropSentryRole(TDropSentryRoleRequest request)
+  public Update onDropSentryRole(TDropSentryRoleRequest request)
       throws SentryPluginException {
     PermissionsUpdate update = new PermissionsUpdate(permSeqNum.incrementAndGet(), false);
     update.addPrivilegeUpdate(PermissionsUpdate.ALL_AUTHZ_OBJ).putToDelPrivileges(
         request.getRoleName(), PermissionsUpdate.ALL_AUTHZ_OBJ);
     update.addRoleUpdate(request.getRoleName()).addToDelGroups(PermissionsUpdate.ALL_GROUPS);
+
     permsUpdater.handleUpdateNotification(update);
     LOGGER.debug("Authz Perm preUpdate [" + update.getSeqNum() + ", " + request.getRoleName() + "]..");
+    return update;
   }
 
   @Override
-  public void onDropSentryPrivilege(TDropPrivilegesRequest request)
+  public Update onDropSentryPrivilege(TDropPrivilegesRequest request)
       throws SentryPluginException {
     PermissionsUpdate update = new PermissionsUpdate(permSeqNum.incrementAndGet(), false);
-    String authzObj = getAuthzObj(request.getAuthorizable());
+    String authzObj = HMSFollower.getAuthzObj(request.getAuthorizable());
     update.addPrivilegeUpdate(authzObj).putToDelPrivileges(
         PermissionsUpdate.ALL_ROLES, PermissionsUpdate.ALL_ROLES);
+
     permsUpdater.handleUpdateNotification(update);
     LOGGER.debug("Authz Perm preUpdate [" + update.getSeqNum() + ", " + authzObj + "]..");
+    return update;
   }
 
   @Override
@@ -427,18 +459,4 @@ public class SentryPlugin implements SentryPolicyStorePlugin, SigUtils.SigListen
     }
     return authzObj == null ? null : authzObj.toLowerCase();
   }
-
-  private String getAuthzObj(TSentryAuthorizable authzble) {
-    String authzObj = null;
-    if (!SentryStore.isNULL(authzble.getDb())) {
-      String dbName = authzble.getDb();
-      String tblName = authzble.getTable();
-      if (SentryStore.isNULL(tblName)) {
-        authzObj = dbName;
-      } else {
-        authzObj = dbName + "." + tblName;
-      }
-    }
-    return authzObj == null ? null : authzObj.toLowerCase();
-  }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SentryPolicyStorePlugin.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SentryPolicyStorePlugin.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SentryPolicyStorePlugin.java
index 2ff715f..5b8a572 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SentryPolicyStorePlugin.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SentryPolicyStorePlugin.java
@@ -28,7 +28,20 @@ import org.apache.sentry.provider.db.service.thrift.TAlterSentryRoleRevokePrivil
 import org.apache.sentry.provider.db.service.thrift.TDropPrivilegesRequest;
 import org.apache.sentry.provider.db.service.thrift.TDropSentryRoleRequest;
 import org.apache.sentry.provider.db.service.thrift.TRenamePrivilegesRequest;
+import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
 
+import java.util.Map;
+
+import static org.apache.sentry.hdfs.Updateable.Update;
+
+/**
+ * Interface for processing delta changes of Sentry permission and generate corresponding
+ * update. The updates will be persisted into Sentry store afterwards along with the actual
+ * operation.
+ *
+ * TODO: SENTRY-1588: add user level privilege change support. e.g. onAlterSentryRoleDeleteUsers,
+ * TODO: onAlterSentryRoleDeleteUsers.
+ */
 public interface SentryPolicyStorePlugin {
 
   @SuppressWarnings("serial")
@@ -43,18 +56,19 @@ public interface SentryPolicyStorePlugin {
 
   void initialize(Configuration conf, SentryStore sentryStore) throws SentryPluginException;
 
-  void onAlterSentryRoleAddGroups(TAlterSentryRoleAddGroupsRequest tRequest) throws SentryPluginException;
-
-  void onAlterSentryRoleDeleteGroups(TAlterSentryRoleDeleteGroupsRequest tRequest) throws SentryPluginException;
+  Update onAlterSentryRoleAddGroups(TAlterSentryRoleAddGroupsRequest tRequest) throws SentryPluginException;
 
-  void onAlterSentryRoleGrantPrivilege(TAlterSentryRoleGrantPrivilegeRequest tRequest) throws SentryPluginException;
+  Update onAlterSentryRoleDeleteGroups(TAlterSentryRoleDeleteGroupsRequest tRequest) throws SentryPluginException;
 
-  void onAlterSentryRoleRevokePrivilege(TAlterSentryRoleRevokePrivilegeRequest tRequest) throws SentryPluginException;
+  void onAlterSentryRoleGrantPrivilege(TAlterSentryRoleGrantPrivilegeRequest tRequest,
+        Map<TSentryPrivilege, Update> privilegesUpdateMap) throws SentryPluginException;
 
-  void onDropSentryRole(TDropSentryRoleRequest tRequest) throws SentryPluginException;
+  void onAlterSentryRoleRevokePrivilege(TAlterSentryRoleRevokePrivilegeRequest tRequest,
+        Map<TSentryPrivilege, Update> privilegesUpdateMap) throws SentryPluginException;
 
-  void onRenameSentryPrivilege(TRenamePrivilegesRequest request) throws SentryPluginException;
+  Update onDropSentryRole(TDropSentryRoleRequest tRequest) throws SentryPluginException;
 
-  void onDropSentryPrivilege(TDropPrivilegesRequest request) throws SentryPluginException;
+  Update onRenameSentryPrivilege(TRenamePrivilegesRequest request) throws SentryPluginException;
 
+  Update onDropSentryPrivilege(TDropPrivilegesRequest request) throws SentryPluginException;
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/DelegateSentryStore.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/DelegateSentryStore.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/DelegateSentryStore.java
index 8e2a6d5..5f396b8 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/DelegateSentryStore.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/generic/service/persistent/DelegateSentryStore.java
@@ -111,7 +111,6 @@ public class DelegateSentryStore implements SentryStoreLayer {
   @Override
   public Object alterRoleDeleteGroups(String component, String role,
       Set<String> groups, String requestor) throws Exception {
-  //called to old sentryStore
     delegate.alterSentryRoleDeleteGroups(role, toTSentryGroups(groups));
     return null;
   }
@@ -121,22 +120,21 @@ public class DelegateSentryStore implements SentryStoreLayer {
       final PrivilegeObject privilege, final String grantorPrincipal)
       throws Exception {
     delegate.getTransactionManager().executeTransactionWithRetry(
-        new TransactionBlock() {
-          public Object execute(PersistenceManager pm) throws Exception {
-            String trimmedRole = toTrimmedLower(role);
-            MSentryRole mRole = getRole(trimmedRole, pm);
-            if (mRole == null) {
-              throw new SentryNoSuchObjectException("Role: " + trimmedRole + " doesn't exist");
-            }
-            /*
-             * check with grant option
-             */
-            grantOptionCheck(privilege, grantorPrincipal, pm);
-
-            privilegeOperator.grantPrivilege(privilege, mRole, pm);
-            return null;
+      new TransactionBlock() {
+        public Object execute(PersistenceManager pm) throws Exception {
+          String trimmedRole = toTrimmedLower(role);
+          MSentryRole mRole = getRole(trimmedRole, pm);
+          if (mRole == null) {
+            throw new SentryNoSuchObjectException("Role: " + trimmedRole + " doesn't exist");
           }
-        });
+
+          // check with grant option
+          grantOptionCheck(privilege, grantorPrincipal, pm);
+
+          privilegeOperator.grantPrivilege(privilege, mRole, pm);
+          return null;
+        }
+      });
     return null;
   }
 
@@ -145,22 +143,21 @@ public class DelegateSentryStore implements SentryStoreLayer {
       final String role, final PrivilegeObject privilege, final String grantorPrincipal)
       throws Exception {
     delegate.getTransactionManager().executeTransactionWithRetry(
-        new TransactionBlock() {
-          public Object execute(PersistenceManager pm) throws Exception {
-            String trimmedRole = toTrimmedLower(role);
-            MSentryRole mRole = getRole(trimmedRole, pm);
-            if (mRole == null) {
-              throw new SentryNoSuchObjectException("Role: " + trimmedRole + " doesn't exist");
-            }
-            /*
-             * check with grant option
-             */
-            grantOptionCheck(privilege, grantorPrincipal, pm);
-
-            privilegeOperator.revokePrivilege(privilege, mRole, pm);
-            return null;
+      new TransactionBlock() {
+        public Object execute(PersistenceManager pm) throws Exception {
+          String trimmedRole = toTrimmedLower(role);
+          MSentryRole mRole = getRole(trimmedRole, pm);
+          if (mRole == null) {
+            throw new SentryNoSuchObjectException("Role: " + trimmedRole + " doesn't exist");
           }
-        });
+
+          // check with grant option
+          grantOptionCheck(privilege, grantorPrincipal, pm);
+
+          privilegeOperator.revokePrivilege(privilege, mRole, pm);
+          return null;
+        }
+      });
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPathChange.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPathChange.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPathChange.java
index b88e7d1..952b11f 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPathChange.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPathChange.java
@@ -18,6 +18,9 @@
 
 package org.apache.sentry.provider.db.service.model;
 
+import org.apache.sentry.hdfs.PathsUpdate;
+import org.apache.thrift.TException;
+
 import javax.jdo.annotations.PersistenceCapable;
 import javax.jdo.annotations.PrimaryKey;
 
@@ -63,24 +66,15 @@ public class MSentryPathChange {
   private String pathChange;
   private long createTimeMs;
 
-  public MSentryPathChange(long changeID, String pathChange, long createTime) {
-    this.changeID = changeID;
-    this.pathChange = pathChange;
-    this.createTimeMs = createTime;
-  }
-
-  public void setCreateTimeMs(long createTimeMs) {
-    this.createTimeMs = createTimeMs;
+  public MSentryPathChange(PathsUpdate pathChange) throws TException {
+    this.pathChange = pathChange.JSONSerialize();
+    this.createTimeMs = System.currentTimeMillis();
   }
 
   public long getCreateTimeMs() {
     return createTimeMs;
   }
 
-  public void setPathChange(String pathChange) {
-    this.pathChange = pathChange;
-  }
-
   public String getPathChange() {
     return pathChange;
   }
@@ -89,13 +83,10 @@ public class MSentryPathChange {
     return changeID;
   }
 
-  public void setChangeID(long changeID) {
-    this.changeID = changeID;
-  }
-
   @Override
   public String toString() {
-    return "MSentryChange [changeID=" + changeID + " , pathChange= " + pathChange + ", createTime=" + createTimeMs +  "]";
+    return "MSentryChange [changeID=" + changeID + " , pathChange= " + pathChange +
+        ", createTime=" + createTimeMs +  "]";
   }
 
   @Override
@@ -126,14 +117,14 @@ public class MSentryPathChange {
       return false;
     }
 
-    if (!pathChange.equals(other.pathChange)) {
+    if (createTimeMs != other.createTimeMs) {
       return false;
     }
 
-    if (createTimeMs != other.createTimeMs) {
-      return false;
+    if (pathChange == null) {
+      return other.pathChange == null;
     }
 
-    return true;
+    return pathChange.equals(other.pathChange);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPermChange.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPermChange.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPermChange.java
index 2ccace0..3662bfd 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPermChange.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPermChange.java
@@ -18,6 +18,9 @@
 
 package org.apache.sentry.provider.db.service.model;
 
+import org.apache.sentry.hdfs.PermissionsUpdate;
+import org.apache.thrift.TException;
+
 import javax.jdo.annotations.PersistenceCapable;
 import javax.jdo.annotations.PrimaryKey;
 
@@ -61,24 +64,15 @@ public class MSentryPermChange {
   private String permChange;
   private long createTimeMs;
 
-  public MSentryPermChange(long changeID, String permChange, long createTimeMs) {
-    this.changeID = changeID;
-    this.permChange = permChange;
-    this.createTimeMs = createTimeMs;
-  }
-
-  public void setCreateTimeMs(long createTimeMs) {
-    this.createTimeMs = createTimeMs;
+  public MSentryPermChange(PermissionsUpdate permChange) throws TException {
+    this.permChange = permChange.JSONSerialize();
+    this.createTimeMs = System.currentTimeMillis();
   }
 
   public long getCreateTimeMs() {
     return createTimeMs;
   }
 
-  public void setPermChange(String permChange) {
-    this.permChange = permChange;
-  }
-
   public String getPermChange() {
     return permChange;
   }
@@ -87,13 +81,10 @@ public class MSentryPermChange {
     return changeID;
   }
 
-  public void setChangeID(long changeID) {
-    this.changeID = changeID;
-  }
-
   @Override
   public String toString() {
-    return "MSentryPermChange [changeID=" + changeID + ", permChange= " + permChange + ", createTimeMs=" + createTimeMs +  "]";
+    return "MSentryPermChange [changeID=" + changeID + ", permChange= " + permChange +
+        ", createTimeMs=" + createTimeMs +  "]";
   }
 
   @Override
@@ -128,10 +119,10 @@ public class MSentryPermChange {
       return false;
     }
 
-    if (!permChange.equals(other.permChange)) {
-      return false;
+    if (permChange == null) {
+      return other.permChange == null;
     }
 
-    return true;
+    return permChange.equals(other.permChange);
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/package.jdo
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/package.jdo b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/package.jdo
index dc8fdbf..94ede1d 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/package.jdo
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/package.jdo
@@ -263,7 +263,7 @@
      </class>
 
      <class name="MSentryPermChange" table="SENTRY_PERM_CHANGE" identity-type="application" detachable="true">
-       <field name="changeID" primary-key="true">
+       <field name="changeID" primary-key="true" value-strategy="increment">
          <column name="CHANGE_ID" jdbc-type="BIGINT" allows-null="false"/>
        </field>
        <field name ="permChange">
@@ -275,7 +275,7 @@
      </class>
 
      <class name="MSentryPathChange" table="SENTRY_PATH_CHANGE" identity-type="application" detachable="true">
-       <field name="changeID" primary-key="true">
+       <field name="changeID" primary-key="true" value-strategy="increment">
          <column name="CHANGE_ID" jdbc-type="BIGINT" allows-null="false"/>
        </field>
        <field name ="pathChange">

http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/DeltaTransactionBlock.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/DeltaTransactionBlock.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/DeltaTransactionBlock.java
new file mode 100644
index 0000000..f590a52
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/DeltaTransactionBlock.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.sentry.provider.db.service.persistent;
+
+import com.google.common.base.Preconditions;
+import org.apache.sentry.core.common.exception.SentryInvalidInputException;
+import org.apache.sentry.hdfs.PathsUpdate;
+import org.apache.sentry.hdfs.PermissionsUpdate;
+import org.apache.sentry.provider.db.service.model.MSentryPathChange;
+import org.apache.sentry.provider.db.service.model.MSentryPermChange;
+import static org.apache.sentry.hdfs.Updateable.Update;
+
+import javax.jdo.PersistenceManager;
+
+/**
+ * DeltaTransactionBlock is an implementation of {@link TransactionBlock}
+ * that persists delta updates for {@link PathsUpdate} or {@link PermissionsUpdate}
+ * into corresponding update table, e.g {@link MSentryPathChange} or
+ * {@link MSentryPermChange}.
+ * <p>
+ * NullPointerException would be thrown if update is null.
+ * {@link SentryInvalidInputException} would be thrown when update is
+ * neither type of PathsUpdate nor PermissionsUpdate, also in the case
+ * update contains a full image. TException would be thrown if Update
+ * cannot be successfully serialized to JSON string.
+ */
+public class DeltaTransactionBlock implements TransactionBlock<Object> {
+  private final Update update;
+
+  public DeltaTransactionBlock(Update update) {
+    this.update = update;
+  }
+
+  @Override
+  public Object execute(PersistenceManager pm) throws Exception {
+    persistUpdate(pm, update);
+    return null;
+  }
+
+  /**
+   * Persist the delta change into corresponding type based on its type.
+   * Atomic increasing primary key changeID by 1.
+   * <p>
+   * NullPointerException would be thrown if update is null.
+   * {@link SentryInvalidInputException} would be thrown when update is
+   * neither type of PathsUpdate nor PermissionsUpdate. Also in the case
+   * update contains a full image.
+   * TException would be thrown if Update cannot be successfully serialized
+   * to JSON string.
+   *
+   * @param pm PersistenceManager
+   * @param update update
+   * @throws Exception
+   */
+  private void persistUpdate(PersistenceManager pm, Update update)
+      throws Exception {
+
+    Preconditions.checkNotNull(update);
+    // persistUpdate cannot handle full image update, instead
+    // it only handles delta updates.
+    if (update.hasFullImage()) {
+      throw new SentryInvalidInputException("Update should be not be a full image.\n");
+    }
+
+    // Persist the update into corresponding tables based on its type.
+    // changeID is the primary key in MSentryPXXXChange table. If same
+    // changeID is trying to be persisted twice, the transaction would
+    // fail.
+    if (update instanceof PermissionsUpdate) {
+      pm.makePersistent(new MSentryPermChange((PermissionsUpdate)update));
+    } else if (update instanceof PathsUpdate) {
+      pm.makePersistent(new MSentryPathChange((PathsUpdate)update));
+    } else {
+      throw new SentryInvalidInputException("Update should be type of either " +
+          "PermissionsUpdate or PathsUpdate.\n");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/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 0712e2c..9b54db1 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
@@ -77,6 +77,8 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
+import static org.apache.sentry.hdfs.Updateable.Update;
+
 /**
  * SentryStore is the data access object for Sentry data. Strings
  * such as role and group names will be normalized to lowercase
@@ -100,6 +102,12 @@ public class SentryStore {
   public static final String GRANT_OPTION = "grantOption";
   public static final String ROLE_NAME = "roleName";
 
+  // Initial change ID for permission/path change. Auto increment
+  // is starting from 1.
+  public static final long INIT_CHANGE_ID = 1L;
+
+  private static final long EMPTY_CHANGE_ID = 0L;
+
   // For counters, representation of the "unknown value"
   private static final long COUNT_VALUE_UNKNOWN = -1;
 
@@ -314,7 +322,7 @@ public class SentryStore {
    */
   public void createSentryRole(final String roleName) throws Exception {
     tm.executeTransactionWithRetry(
-        new TransactionBlock() {
+        new TransactionBlock<Object>() {
           public Object execute(PersistenceManager pm) throws Exception {
             String trimmedRoleName = trimAndLower(roleName);
             if (getRole(pm, trimmedRoleName) != null) {
@@ -410,7 +418,7 @@ public class SentryStore {
   void clearAllTables() {
     try {
       tm.executeTransaction(
-          new TransactionBlock() {
+          new TransactionBlock<Object>() {
             public Object execute(PersistenceManager pm) throws Exception {
               pm.newQuery(MSentryRole.class).deletePersistentAll();
               pm.newQuery(MSentryGroup.class).deletePersistentAll();
@@ -426,20 +434,40 @@ public class SentryStore {
   }
 
   /**
-   * Grant privilege for a role
+   * Alter a given sentry role to grant a privilege.
+   *
    * @param grantorPrincipal User name
-   * @param roleName Role name
-   * @param privilege Privilege to grant
+   * @param roleName the given role name
+   * @param privilege the given privilege
    * @throws Exception
    */
-  void alterSentryRoleGrantPrivilege(String grantorPrincipal,
-      String roleName, TSentryPrivilege privilege) throws Exception {
-    alterSentryRoleGrantPrivileges(grantorPrincipal, roleName,
-            Sets.newHashSet(privilege));
+  void alterSentryRoleGrantPrivilege(final String grantorPrincipal,
+      final String roleName, final TSentryPrivilege privilege) throws Exception {
+
+    tm.executeTransactionWithRetry(
+      new TransactionBlock<Object>() {
+        public Object execute(PersistenceManager pm) throws Exception {
+          String trimmedRoleName = trimAndLower(roleName);
+          // first do grant check
+          grantOptionCheck(pm, grantorPrincipal, privilege);
+
+          // Alter sentry Role and grant Privilege.
+          MSentryPrivilege mPrivilege = alterSentryRoleGrantPrivilegeCore(
+            pm, trimmedRoleName, privilege);
+
+          if (mPrivilege != null) {
+            // update the privilege to be the one actually updated.
+            convertToTSentryPrivilege(mPrivilege, privilege);
+          }
+          return null;
+        }
+      });
   }
 
   /**
-   * Grant multiple privileges
+   * Alter a given sentry role to grant a set of privileges.
+   * Internally calls alterSentryRoleGrantPrivilege.
+   *
    * @param grantorPrincipal User name
    * @param roleName Role name
    * @param privileges Set of privileges
@@ -447,22 +475,71 @@ public class SentryStore {
    */
   public void alterSentryRoleGrantPrivileges(final String grantorPrincipal,
       final String roleName, final Set<TSentryPrivilege> privileges) throws Exception {
-    tm.executeTransactionWithRetry(
-        new TransactionBlock() {
-          public Object execute(PersistenceManager pm) throws Exception {
-            String trimmedRoleName = trimAndLower(roleName);
-            for (TSentryPrivilege privilege : privileges) {
-              // first do grant check
-              grantOptionCheck(pm, grantorPrincipal, privilege);
-              MSentryPrivilege mPrivilege = alterSentryRoleGrantPrivilegeCore(
-                  pm, trimmedRoleName, privilege);
-              if (mPrivilege != null) {
-                convertToTSentryPrivilege(mPrivilege, privilege);
-              }
-            }
-            return null;
-          }
-        });
+    for (TSentryPrivilege privilege : privileges) {
+      alterSentryRoleGrantPrivilege(grantorPrincipal, roleName, privilege);
+    }
+  }
+
+  /**
+   * Alter a given sentry role to grant a privilege, as well as persist the corresponding
+   * permission change to MSentryPermChange table in a single transaction.
+   *
+   * @param grantorPrincipal User name
+   * @param roleName the given role name
+   * @param privilege the given privilege
+   * @param update the corresponding permission delta update.
+   * @throws Exception
+   *
+   */
+  void alterSentryRoleGrantPrivilege(final String grantorPrincipal,
+      final String roleName, final TSentryPrivilege privilege,
+      final Update update) throws Exception {
+
+    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+      public Object execute(PersistenceManager pm) throws Exception {
+        String trimmedRoleName = trimAndLower(roleName);
+        // first do grant check
+        grantOptionCheck(pm, grantorPrincipal, privilege);
+
+        // Alter sentry Role and grant Privilege.
+        MSentryPrivilege mPrivilege = alterSentryRoleGrantPrivilegeCore(pm,
+          trimmedRoleName, privilege);
+
+        if (mPrivilege != null) {
+          // update the privilege to be the one actually updated.
+          convertToTSentryPrivilege(mPrivilege, privilege);
+        }
+        return null;
+      }
+    });
+  }
+
+  /**
+   * Alter a given sentry role to grant a set of privileges, as well as persist the
+   * corresponding permission change to MSentryPermChange table in a single transaction.
+   * Internally calls alterSentryRoleGrantPrivilege.
+   *
+   * @param grantorPrincipal User name
+   * @param roleName the given role name
+   * @param privileges a Set of privileges
+   * @param privilegesUpdateMap the corresponding <privilege, DeltaTransactionBlock> map
+   * @throws Exception
+   *
+   */
+  public void alterSentryRoleGrantPrivileges(final String grantorPrincipal,
+      final String roleName, final Set<TSentryPrivilege> privileges,
+      final Map<TSentryPrivilege, Update> privilegesUpdateMap) throws Exception {
+
+    Preconditions.checkNotNull(privilegesUpdateMap);
+    for (TSentryPrivilege privilege : privileges) {
+      Update update = privilegesUpdateMap.get(privilege);
+      if (update != null) {
+        alterSentryRoleGrantPrivilege(grantorPrincipal, roleName, privilege,
+          update);
+      } else {
+        alterSentryRoleGrantPrivilege(grantorPrincipal, roleName, privilege);
+      }
+    }
   }
 
   private MSentryPrivilege alterSentryRoleGrantPrivilegeCore(PersistenceManager pm,
@@ -522,27 +599,102 @@ public class SentryStore {
     return mPrivilege;
   }
 
-  void alterSentryRoleRevokePrivilege(String grantorPrincipal,
-      String roleName, TSentryPrivilege tPrivilege) throws Exception {
-    alterSentryRoleRevokePrivileges(grantorPrincipal, roleName,
-            Sets.newHashSet(tPrivilege));
+  /**
+  * Alter a given sentry role to revoke a privilege.
+  *
+  * @param grantorPrincipal User name
+  * @param roleName the given role name
+  * @param tPrivilege the given privilege
+  * @throws Exception
+  *
+  */
+  void alterSentryRoleRevokePrivilege(final String grantorPrincipal,
+      final String roleName, final TSentryPrivilege tPrivilege) throws Exception {
+
+    tm.executeTransactionWithRetry(
+      new TransactionBlock<Object>() {
+        public Object execute(PersistenceManager pm) throws Exception {
+          String trimmedRoleName = safeTrimLower(roleName);
+          // first do revoke check
+          grantOptionCheck(pm, grantorPrincipal, tPrivilege);
+
+          alterSentryRoleRevokePrivilegeCore(pm, trimmedRoleName, tPrivilege);
+          return null;
+        }
+      });
   }
 
+  /**
+   * Alter a given sentry role to revoke a set of privileges.
+   * Internally calls alterSentryRoleRevokePrivilege.
+   *
+   * @param grantorPrincipal User name
+   * @param roleName the given role name
+   * @param tPrivileges a Set of privileges
+   * @throws Exception
+   *
+   */
   public void alterSentryRoleRevokePrivileges(final String grantorPrincipal,
       final String roleName, final Set<TSentryPrivilege> tPrivileges) throws Exception {
-    tm.executeTransactionWithRetry(
-        new TransactionBlock() {
-          public Object execute(PersistenceManager pm) throws Exception {
-            String trimmedRoleName = safeTrimLower(roleName);
-            for (TSentryPrivilege tPrivilege : tPrivileges) {
-              // first do revoke check
-              grantOptionCheck(pm, grantorPrincipal, tPrivilege);
+    for (TSentryPrivilege tPrivilege : tPrivileges) {
+      alterSentryRoleRevokePrivilege(grantorPrincipal, roleName, tPrivilege);
+    }
+  }
 
-              alterSentryRoleRevokePrivilegeCore(pm, trimmedRoleName, tPrivilege);
-            }
-            return null;
-          }
-        });
+  /**
+   * Alter a given sentry role to revoke a privilege, as well as persist the corresponding
+   * permission change to MSentryPermChange table in a single transaction.
+   *
+   * @param grantorPrincipal User name
+   * @param roleName the given role name
+   * @param tPrivilege the given privilege
+   * @param update the corresponding permission delta update transaction block
+   * @throws Exception
+   *
+   */
+  void alterSentryRoleRevokePrivilege(final String grantorPrincipal,
+      final String roleName, final TSentryPrivilege tPrivilege,
+      final Update update) throws Exception {
+    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+      public Object execute(PersistenceManager pm) throws Exception {
+        String trimmedRoleName = safeTrimLower(roleName);
+        // first do revoke check
+        grantOptionCheck(pm, grantorPrincipal, tPrivilege);
+
+        alterSentryRoleRevokePrivilegeCore(pm, trimmedRoleName, tPrivilege);
+        return null;
+      }
+    });
+  }
+
+  /**
+   * Alter a given sentry role to revoke a set of privileges, as well as persist the
+   * corresponding permission change to MSentryPermChange table in a single transaction.
+   * Internally calls alterSentryRoleRevokePrivilege.
+   *
+   * @param grantorPrincipal User name
+   * @param roleName the given role name
+   * @param tPrivileges a Set of privileges
+   * @param privilegesUpdateMap the corresponding <privilege, Update> map
+   * @throws Exception
+   *
+   */
+  public void alterSentryRoleRevokePrivileges(final String grantorPrincipal,
+      final String roleName, 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) {
+        alterSentryRoleRevokePrivilege(grantorPrincipal, roleName,
+          tPrivilege, update);
+      } else {
+        alterSentryRoleRevokePrivilege(grantorPrincipal, roleName,
+          tPrivilege);
+      }
+    }
   }
 
   private void alterSentryRoleRevokePrivilegeCore(PersistenceManager pm,
@@ -795,9 +947,15 @@ public class SentryStore {
     return (MSentryPrivilege)query.executeWithMap(paramBuilder.getArguments());
   }
 
+  /**
+   * Drop a given sentry role.
+   *
+   * @param roleName the given role name
+   * @throws Exception
+   */
   public void dropSentryRole(final String roleName) throws Exception {
     tm.executeTransactionWithRetry(
-        new TransactionBlock() {
+        new TransactionBlock<Object>() {
           public Object execute(PersistenceManager pm) throws Exception {
             dropSentryRoleCore(pm, roleName);
             return null;
@@ -805,6 +963,25 @@ public class SentryStore {
         });
   }
 
+  /**
+   * Drop a given sentry role. As well as persist the corresponding
+   * permission change to MSentryPermChange table in a single transaction.
+   *
+   * @param roleName the given role name
+   * @param update the corresponding permission delta update
+   * @throws Exception
+   */
+  public void dropSentryRole(final String roleName,
+      final Update update) throws Exception {
+
+    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+      public Object execute(PersistenceManager pm) throws Exception {
+        dropSentryRoleCore(pm, roleName);
+        return null;
+      }
+    });
+  }
+
   private void dropSentryRoleCore(PersistenceManager pm, String roleName)
       throws SentryNoSuchObjectException {
     String lRoleName = trimAndLower(roleName);
@@ -820,10 +997,18 @@ public class SentryStore {
     pm.deletePersistent(sentryRole);
   }
 
+  /**
+   * Assign a given role to a set of groups.
+   *
+   * @param grantorPrincipal grantorPrincipal currently is not used.
+   * @param roleName the role to be assigned to the groups.
+   * @param groupNames the list of groups to be added to the role,
+   * @throws Exception
+   */
   public void alterSentryRoleAddGroups(final String grantorPrincipal,
       final String roleName, final Set<TSentryGroup> groupNames) throws Exception {
     tm.executeTransactionWithRetry(
-        new TransactionBlock() {
+        new TransactionBlock<Object>() {
           public Object execute(PersistenceManager pm) throws Exception {
             alterSentryRoleAddGroupsCore(pm, roleName, groupNames);
             return null;
@@ -831,13 +1016,39 @@ public class SentryStore {
         });
   }
 
+  /**
+   * Assign a given role to a set of groups. As well as persist the corresponding
+   * permission change to MSentryPermChange table in a single transaction.
+   *
+   * @param grantorPrincipal grantorPrincipal currently is not used.
+   * @param roleName the role to be assigned to the groups.
+   * @param groupNames the list of groups to be added to the role,
+   * @param update the corresponding permission delta update
+   * @throws Exception
+   */
+  public void alterSentryRoleAddGroups(final String grantorPrincipal,
+      final String roleName, final Set<TSentryGroup> groupNames,
+      final Update update) throws Exception {
+
+    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+      public Object execute(PersistenceManager pm) throws Exception {
+        alterSentryRoleAddGroupsCore(pm, roleName, groupNames);
+        return null;
+      }
+    });
+  }
+
   private void alterSentryRoleAddGroupsCore(PersistenceManager pm, String roleName,
       Set<TSentryGroup> groupNames) throws SentryNoSuchObjectException {
+
+    // All role names are stored in lowercase.
     String lRoleName = trimAndLower(roleName);
     MSentryRole role = getRole(pm, lRoleName);
     if (role == null) {
       throw noSuchRole(lRoleName);
     }
+
+    // Add the group to the specified role if it does not belong to the role yet.
     Query query = pm.newQuery(MSentryGroup.class);
     query.setFilter("this.groupName == :groupName");
     query.setUnique(true);
@@ -857,7 +1068,7 @@ public class SentryStore {
   public void alterSentryRoleAddUsers(final String roleName,
       final Set<String> userNames) throws Exception {
     tm.executeTransactionWithRetry(
-        new TransactionBlock() {
+        new TransactionBlock<Object>() {
           public Object execute(PersistenceManager pm) throws Exception {
             alterSentryRoleAddUsersCore(pm, roleName, userNames);
             return null;
@@ -891,7 +1102,7 @@ public class SentryStore {
   public void alterSentryRoleDeleteUsers(final String roleName,
       final Set<String> userNames) throws Exception {
     tm.executeTransactionWithRetry(
-        new TransactionBlock() {
+        new TransactionBlock<Object>() {
           public Object execute(PersistenceManager pm) throws Exception {
             String trimmedRoleName = trimAndLower(roleName);
             MSentryRole role = getRole(pm, trimmedRoleName);
@@ -917,10 +1128,17 @@ public class SentryStore {
         });
   }
 
+  /**
+   * Revoke a given role to a set of groups.
+   *
+   * @param roleName the role to be assigned to the groups.
+   * @param groupNames the list of groups to be added to the role,
+   * @throws Exception
+   */
   public void alterSentryRoleDeleteGroups(final String roleName,
       final Set<TSentryGroup> groupNames) throws Exception {
     tm.executeTransactionWithRetry(
-        new TransactionBlock() {
+        new TransactionBlock<Object>() {
           public Object execute(PersistenceManager pm) throws Exception {
             String trimmedRoleName = trimAndLower(roleName);
             MSentryRole role = getRole(pm, trimmedRoleName);
@@ -945,6 +1163,45 @@ public class SentryStore {
         });
   }
 
+  /**
+   * Revoke a given role to a set of groups. As well as persist the corresponding
+   * permission change to MSentryPermChange table in a single transaction.
+   *
+   * @param roleName the role to be assigned to the groups.
+   * @param groupNames the list of groups to be added to the role,
+   * @param update the corresponding permission delta update
+   * @throws Exception
+   */
+  public void alterSentryRoleDeleteGroups(final String roleName,
+      final Set<TSentryGroup> groupNames, final Update update)
+          throws Exception {
+    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+      public Object execute(PersistenceManager pm) throws Exception {
+        String trimmedRoleName = trimAndLower(roleName);
+        MSentryRole role = getRole(pm, trimmedRoleName);
+        if (role == null) {
+          throw noSuchRole(trimmedRoleName);
+        }
+
+        // Remove the group from the specified role if it belongs to the role.
+        Query query = pm.newQuery(MSentryGroup.class);
+        query.setFilter("this.groupName == :groupName");
+        query.setUnique(true);
+        List<MSentryGroup> groups = Lists.newArrayList();
+        for (TSentryGroup tGroup : groupNames) {
+          String groupName = tGroup.getGroupName().trim();
+          MSentryGroup group = (MSentryGroup) query.execute(groupName);
+          if (group != null) {
+            group.removeRole(role);
+            groups.add(group);
+          }
+        }
+        pm.makePersistentAll(groups);
+        return null;
+      }
+    });
+  }
+
   @VisibleForTesting
   MSentryRole getMSentryRoleByName(final String roleName) throws Exception {
     return tm.executeTransaction(
@@ -1493,7 +1750,7 @@ public class SentryStore {
   void setSentryVersion(final String newVersion, final String verComment)
       throws Exception {
     tm.executeTransaction(
-        new TransactionBlock() {
+        new TransactionBlock<Object>() {
           public Object execute(PersistenceManager pm) throws Exception {
             MSentryVersion mVersion;
             try {
@@ -1545,14 +1802,19 @@ public class SentryStore {
   }
 
   /**
-   * Drop given privilege from all roles
+   * Drop the given privilege from all roles.
+   *
+   * @param tAuthorizable the given authorizable object.
+   * @throws Exception
    */
   public void dropPrivilege(final TSentryAuthorizable tAuthorizable) throws Exception {
     tm.executeTransactionWithRetry(
-        new TransactionBlock() {
+        new TransactionBlock<Object>() {
           public Object execute(PersistenceManager pm) throws Exception {
 
+            // Drop the give privilege for all possible actions from all roles.
             TSentryPrivilege tPrivilege = toSentryPrivilege(tAuthorizable);
+
             try {
               if (isMultiActionsSupported(tPrivilege)) {
                 for (String privilegeAction : ALL_ACTIONS) {
@@ -1572,20 +1834,58 @@ public class SentryStore {
   }
 
   /**
-   * Rename given privilege from all roles drop the old privilege and create the new one
-   * @param tAuthorizable
-   * @param newTAuthorizable
+   * Drop the given privilege from all roles. As well as persist the corresponding
+   * permission change to MSentryPermChange table in a single transaction.
+   *
+   * @param tAuthorizable the given authorizable object.
+   * @param update the corresponding permission delta update.
+   * @throws Exception
+   */
+  public void dropPrivilege(final TSentryAuthorizable tAuthorizable,
+      final Update update) throws Exception {
+
+    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+      public Object execute(PersistenceManager pm) throws Exception {
+
+        // Drop the give privilege for all possible actions from all roles.
+        TSentryPrivilege tPrivilege = toSentryPrivilege(tAuthorizable);
+
+        try {
+          if (isMultiActionsSupported(tPrivilege)) {
+            for (String privilegeAction : ALL_ACTIONS) {
+              tPrivilege.setAction(privilegeAction);
+              dropPrivilegeForAllRoles(pm, new TSentryPrivilege(tPrivilege));
+            }
+          } else {
+            dropPrivilegeForAllRoles(pm, new TSentryPrivilege(tPrivilege));
+          }
+        } catch (JDODataStoreException e) {
+          throw new SentryInvalidInputException("Failed to get privileges: "
+          + e.getMessage());
+        }
+        return null;
+      }
+    });
+  }
+
+  /**
+   * Rename the privilege for all roles. Drop the old privilege name and create the new one.
+   *
+   * @param oldTAuthorizable the old authorizable name needs to be renamed.
+   * @param newTAuthorizable the new authorizable name
    * @throws SentryNoSuchObjectException
    * @throws SentryInvalidInputException
    */
-  public void renamePrivilege(final TSentryAuthorizable tAuthorizable,
+  public void renamePrivilege(final TSentryAuthorizable oldTAuthorizable,
       final TSentryAuthorizable newTAuthorizable) throws Exception {
     tm.executeTransactionWithRetry(
-        new TransactionBlock() {
+        new TransactionBlock<Object>() {
           public Object execute(PersistenceManager pm) throws Exception {
 
-            TSentryPrivilege tPrivilege = toSentryPrivilege(tAuthorizable);
+            // Drop the give privilege for all possible actions from all roles.
+            TSentryPrivilege tPrivilege = toSentryPrivilege(oldTAuthorizable);
             TSentryPrivilege newPrivilege = toSentryPrivilege(newTAuthorizable);
+
             try {
               // In case of tables or DBs, check all actions
               if (isMultiActionsSupported(tPrivilege)) {
@@ -1606,6 +1906,48 @@ public class SentryStore {
         });
   }
 
+  /**
+   * Rename the privilege for all roles. Drop the old privilege name and create the new one.
+   * As well as persist the corresponding permission change to MSentryPermChange table in a
+   * single transaction.
+   *
+   * @param oldTAuthorizable the old authorizable name needs to be renamed.
+   * @param newTAuthorizable the new authorizable name
+   * @param update the corresponding permission delta update.
+   * @throws SentryNoSuchObjectException
+   * @throws SentryInvalidInputException
+   */
+  public void renamePrivilege(final TSentryAuthorizable oldTAuthorizable,
+      final TSentryAuthorizable newTAuthorizable, final Update update)
+        throws Exception {
+
+    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+      public Object execute(PersistenceManager pm) throws Exception {
+
+        // Drop the give privilege for all possible actions from all roles.
+        TSentryPrivilege tPrivilege = toSentryPrivilege(oldTAuthorizable);
+        TSentryPrivilege newPrivilege = toSentryPrivilege(newTAuthorizable);
+
+        try {
+          // In case of tables or DBs, check all actions
+          if (isMultiActionsSupported(tPrivilege)) {
+            for (String privilegeAction : ALL_ACTIONS) {
+              tPrivilege.setAction(privilegeAction);
+              newPrivilege.setAction(privilegeAction);
+              renamePrivilegeForAllRoles(pm, tPrivilege, newPrivilege);
+            }
+          } else {
+            renamePrivilegeForAllRoles(pm, tPrivilege, newPrivilege);
+          }
+        } catch (JDODataStoreException e) {
+          throw new SentryInvalidInputException("Failed to get privileges: "
+          + e.getMessage());
+        }
+        return null;
+      }
+    });
+  }
+
   // Currently INSERT/SELECT/ALL are supported for Table and DB level privileges
   private boolean isMultiActionsSupported(TSentryPrivilege tPrivilege) {
     return tPrivilege.getDbName() != null;
@@ -1898,7 +2240,7 @@ public class SentryStore {
     Map<String, Set<String>> result = new HashMap<>();
     try {
       result = (Map<String, Set<String>>) tm.executeTransaction(
-          new TransactionBlock() {
+          new TransactionBlock<Object>() {
             public Object execute(PersistenceManager pm) throws Exception {
               Map<String, Set<String>> retVal = new HashMap<>();
               Query query = pm.newQuery(MAuthzPathsMapping.class);
@@ -1918,7 +2260,7 @@ public class SentryStore {
   public void createAuthzPathsMapping(final String hiveObj,
       final Set<String> paths) throws Exception {
     tm.executeTransactionWithRetry(
-        new TransactionBlock() {
+        new TransactionBlock<Object>() {
           public Object execute(PersistenceManager pm) throws Exception {
             createAuthzPathsMappingCore(pm, hiveObj, paths);
             return null;
@@ -2440,7 +2782,7 @@ public class SentryStore {
   public void importSentryMetaData(final TSentryMappingData tSentryMappingData,
       final boolean isOverwriteForRole) throws Exception {
     tm.executeTransaction(
-        new TransactionBlock() {
+        new TransactionBlock<Object>() {
           public Object execute(PersistenceManager pm) throws Exception {
             TSentryMappingData mappingData = lowercaseRoleName(tSentryMappingData);
             Set<String> roleNames = getAllRoleNamesCore(pm);
@@ -2620,6 +2962,15 @@ public class SentryStore {
   }
 
   /**
+   * Return exception for nonexistent update
+   * @param changeID change ID
+   * @return SentryNoSuchObjectException with appropriate message
+   */
+  private SentryNoSuchObjectException noSuchUpdate(final long changeID) {
+    return new SentryNoSuchObjectException("nonexistent update + " + changeID);
+  }
+
+  /**
    * Add common filter for set of roles
    * @param query Query used for search
    * @param paramBuilder paramBuilder for parameters
@@ -2926,4 +3277,109 @@ public class SentryStore {
       return this;
     }
   }
+
+  /**
+   * Get the last processed perm change ID.
+   *
+   * @param pm the PersistenceManager
+   * @return the last processed perm changedID
+   */
+  private long getLastProcessedPermChangeIDCore(PersistenceManager pm) {
+    Query query = pm.newQuery(MSentryPermChange.class);
+    query.setResult("max(this.changeID)");
+    Long changeID = (Long) query.execute();
+    if (changeID == null) {
+      return EMPTY_CHANGE_ID;
+    } else {
+      return changeID;
+    }
+  }
+
+  /**
+   * Get the MSentryPermChange object by ChangeID. Internally invoke
+   * getLastProcessedPermChangeIDCore().
+   *
+   * @return MSentryPermChange
+   */
+  @VisibleForTesting
+  long getLastProcessedPermChangeID() throws Exception {
+    return tm.executeTransaction(
+      new TransactionBlock<Long>() {
+        public Long execute(PersistenceManager pm) throws Exception {
+          return getLastProcessedPermChangeIDCore(pm);
+        }
+      });
+  }
+
+  /**
+   * Get the MSentryPermChange object by ChangeID.
+   *
+   * @param changeID the given changeID.
+   * @return MSentryPermChange
+   */
+  public MSentryPermChange getMSentryPermChangeByID(final long changeID) throws Exception {
+    return (MSentryPermChange) tm.executeTransaction(
+      new TransactionBlock<Object>() {
+        public Object execute(PersistenceManager pm) throws Exception {
+          Query query = pm.newQuery(MSentryPermChange.class);
+          query.setFilter("this.changeID == t");
+          query.declareParameters("long t");
+          List<MSentryPermChange> permChanges = (List<MSentryPermChange>)query.execute(changeID);
+          if (permChanges == null) {
+            noSuchUpdate(changeID);
+          } else if (permChanges.size() > 1) {
+            throw new Exception("Inconsistent permission delta: " + permChanges.size()
+                + " permissions for the same id, " + changeID);
+          }
+
+          return permChanges.get(0);
+        }
+      });
+  }
+
+  /**
+   * Get the MSentryPathChange object by ChangeID.
+   */
+  public MSentryPathChange getMSentryPathChangeByID(final long changeID) throws Exception {
+    return (MSentryPathChange) tm.executeTransaction(
+      new TransactionBlock<Object>() {
+        public Object execute(PersistenceManager pm) throws Exception {
+          Query query = pm.newQuery(MSentryPathChange.class);
+          query.setFilter("this.changeID == t");
+          query.declareParameters("long t");
+          List<MSentryPathChange> pathChanges = (List<MSentryPathChange>)query.execute(changeID);
+          if (pathChanges == null) {
+            noSuchUpdate(changeID);
+          } else if (pathChanges.size() > 1) {
+            throw new Exception("Inconsistent path delta: " + pathChanges.size()
+                + " paths for the same id, " + changeID);
+          }
+
+          return pathChanges.get(0);
+        }
+      });
+  }
+
+  /**
+   * Execute Perm/Path UpdateTransaction and corresponding actual
+   * action transaction, e.g dropSentryRole, in a single transaction.
+   * The order of the transaction does not matter because there is no
+   * any return value.
+   * <p>
+   * Failure in any TransactionBlock would cause the whole transaction
+   * to fail.
+   *
+   * @param deltaTransactionBlock
+   * @param transactionBlock
+   * @throws Exception
+   */
+  private void execute(DeltaTransactionBlock deltaTransactionBlock,
+        TransactionBlock<Object> transactionBlock) throws Exception {
+    List<TransactionBlock<Object>> tbs = Lists.newArrayList();
+    if (deltaTransactionBlock != null) {
+      tbs.add(deltaTransactionBlock);
+    }
+    tbs.add(transactionBlock);
+    tm.executeTransactionBlocksWithRetry(tbs);
+  }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/TransactionManager.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/TransactionManager.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/TransactionManager.java
index 6428a0c..fb7c40a 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/TransactionManager.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/TransactionManager.java
@@ -34,12 +34,14 @@ import javax.jdo.Transaction;
 
 import org.apache.sentry.provider.db.service.thrift.SentryMetrics;
 
+import java.util.List;
 
 /**
  * TransactionManager is used for executing the database transaction, it supports
  * the transaction with retry mechanism for the unexpected exceptions,
  * except <em>SentryUserExceptions</em>, eg, <em>SentryNoSuchObjectException</em>,
- * <em>SentryAlreadyExistsException</em> etc. <p>
+ * <em>SentryAlreadyExistsException</em> etc. For <em>SentryUserExceptions</em>,
+ * will simply throw the exception without retry<p>
  *
  * The purpose of the class is to separate all transaction housekeeping (opening
  * transaction, rolling back failed transactions) from the actual transaction
@@ -99,7 +101,8 @@ public class TransactionManager {
    * Execute some code as a single transaction, the code in tb.execute()
    * should not start new transaction or manipulate transactions with the
    * PersistenceManager.
-   * @param tb transaction block with code to execute
+   *
+   * @param tb transaction block with code to be executed
    * @return Object with the result of tb.execute()
    */
   public <T> T executeTransaction(TransactionBlock<T> tb) throws Exception {
@@ -129,7 +132,46 @@ public class TransactionManager {
   }
 
   /**
-   * Execute some code as a single transaction with retry mechanism
+   * Execute a list of TransactionBlock code as a single transaction.
+   * The code in tb.execute() should not start new transaction or
+   * manipulate transactions with the PersistenceManager. It returns
+   * the result of the last transaction block execution.
+   *
+   * @param tbs transaction blocks with code to be executed
+   * @return the result of the last result of tb.execute()
+   */
+  public <T> T executeTransaction(Iterable<TransactionBlock<T>> tbs) throws Exception {
+    final Timer.Context context = transactionTimer.time();
+    try (PersistenceManager pm = pmf.getPersistenceManager()) {
+      Transaction transaction = pm.currentTransaction();
+      transaction.begin();
+      try {
+        T result = null;
+        for (TransactionBlock<T> tb : tbs) {
+          result = tb.execute(pm);
+        }
+        transaction.commit();
+        return result;
+      } catch (Exception e) {
+        // Count total failed transactions
+        failedTransactionsCount.inc();
+        // Count specific exceptions
+        SentryMetrics.getInstance().getCounter(name(TransactionManager.class,
+            "exception", e.getClass().getSimpleName())).inc();
+        // Re-throw the exception
+        throw e;
+      } finally {
+        context.stop();
+        if (transaction.isActive()) {
+          transaction.rollback();
+        }
+      }
+    }
+  }
+
+  /**
+   * Execute some code as a single transaction with retry mechanism.
+   *
    * @param tb transaction block with code to execute
    * @return Object with the result of tb.execute()
    */
@@ -139,20 +181,55 @@ public class TransactionManager {
     while (retryNum < transactionRetryMax) {
       try {
         return executeTransaction(tb);
-      } catch (Exception e) {
+      } catch (SentryUserException e) {
         // throw the sentry exception without retry
-        if (e instanceof SentryUserException) {
-          throw e;
+        throw e;
+      } catch (Exception e) {
+        retryNum++;
+        if (retryNum >= transactionRetryMax) {
+          String message = "The transaction has reached max retry numbe, r"
+              + e.getMessage();
+          LOGGER.error(message, e);
+          throw new Exception(message, e);
         }
+        retryCount.inc();
+        LOGGER.warn("Exception during transaction execution, retrying "
+            + retryNum + "times. The max retry num is: " + transactionRetryMax, e);
+        Thread.sleep(retryWaitTimeMills);
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Execute a list of TransactionBlock code as a single transaction.
+   * If any of the TransactionBlock fail, all the TransactionBlocks would
+   * retry. It returns the result of the last transaction block
+   * execution.
+   *
+   * @param tbs a list of transaction blocks with code to be executed.
+   * @return the result of the last transaction block execution.
+   */
+  public <T> T executeTransactionBlocksWithRetry(Iterable<TransactionBlock<T>> tbs)
+          throws Exception {
+    int retryNum = 0;
+    while (retryNum < transactionRetryMax) {
+      try {
+        return executeTransaction(tbs);
+      } catch (SentryUserException e) {
+        // throw the sentry exception without retry
+        throw e;
+      } catch (Exception e) {
         retryNum++;
         if (retryNum >= transactionRetryMax) {
-          String message = "The transaction has reached max retry number, will not retry again.";
+          String message = "The transaction has reached max retry number, "
+              + e.getMessage();
           LOGGER.error(message, e);
           throw new Exception(message, e);
         }
         retryCount.inc();
-        LOGGER.warn("Exception is thrown, retry the transaction, current retry num is:"
-            + retryNum + ", the max retry num is: " + transactionRetryMax, e);
+        LOGGER.warn("Exception during transaction execution, retrying "
+            + retryNum + "times. The max retry num is: " + transactionRetryMax, e);
         Thread.sleep(retryWaitTimeMills);
       }
     }

http://git-wip-us.apache.org/repos/asf/sentry/blob/2911c532/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 5121740..7fc3ca8 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
@@ -20,6 +20,7 @@ package org.apache.sentry.provider.db.service.thrift;
 
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
@@ -67,6 +68,8 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
+import static org.apache.sentry.hdfs.Updateable.Update;
+
 @SuppressWarnings("unused")
 public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
   private static final Logger LOGGER = LoggerFactory.getLogger(SentryPolicyStoreProcessor.class);
@@ -74,8 +77,6 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
 
   public static final String SENTRY_POLICY_SERVICE_NAME = "SentryPolicyService";
 
-  public static volatile SentryPolicyStoreProcessor instance;
-
   private final String name;
   private final Configuration conf;
   private final SentryStore sentryStore;
@@ -109,9 +110,6 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       plugin.initialize(conf, sentryStore);
       sentryPlugins.add(plugin);
     }
-    if (instance == null) {
-      instance = this;
-    }
     initMetrics();
   }
 
@@ -244,8 +242,22 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       if (request.isSetPrivilege()) {
         request.setPrivileges(Sets.newHashSet(request.getPrivilege()));
       }
-      sentryStore.alterSentryRoleGrantPrivileges(request.getRequestorUserName(),
-          request.getRoleName(), request.getPrivileges());
+
+      // TODO: now only has SentryPlugin. Once add more SentryPolicyStorePlugins,
+      // TODO: need to differentiate the updates for different Plugins.
+      Preconditions.checkState(sentryPlugins.size() <= 1);
+      Map<TSentryPrivilege, Update> privilegesUpdateMap = new HashMap<>();
+      for (SentryPolicyStorePlugin plugin : sentryPlugins) {
+        plugin.onAlterSentryRoleGrantPrivilege(request, privilegesUpdateMap);
+      }
+
+      if (!privilegesUpdateMap.isEmpty()) {
+        sentryStore.alterSentryRoleGrantPrivileges(request.getRequestorUserName(),
+            request.getRoleName(), request.getPrivileges(), privilegesUpdateMap);
+      } else {
+        sentryStore.alterSentryRoleGrantPrivileges(request.getRequestorUserName(),
+            request.getRoleName(), request.getPrivileges());
+      }
       response.setStatus(Status.OK());
       response.setPrivileges(request.getPrivileges());
       // Maintain compatibility for old API: Set privilege field to response
@@ -254,9 +266,6 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       }
       notificationHandlerInvoker.alter_sentry_role_grant_privilege(request,
               response);
-      for (SentryPolicyStorePlugin plugin : sentryPlugins) {
-        plugin.onAlterSentryRoleGrantPrivilege(request);
-      }
     } catch (SentryNoSuchObjectException e) {
       String msg = "Role: " + request.getRoleName() + " doesn't exist";
       LOGGER.error(msg, e);
@@ -308,14 +317,25 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       if (request.isSetPrivilege()) {
         request.setPrivileges(Sets.newHashSet(request.getPrivilege()));
       }
-      sentryStore.alterSentryRoleRevokePrivileges(request.getRequestorUserName(),
-          request.getRoleName(), request.getPrivileges());
+
+      // TODO: now only has SentryPlugin. Once add more SentryPolicyStorePlugins,
+      // TODO: need to differentiate the updates for different Plugins.
+      Preconditions.checkState(sentryPlugins.size() <= 1);
+      Map<TSentryPrivilege, Update> privilegesUpdateMap = new HashMap<>();
+      for (SentryPolicyStorePlugin plugin : sentryPlugins) {
+        plugin.onAlterSentryRoleRevokePrivilege(request, privilegesUpdateMap);
+      }
+
+      if (!privilegesUpdateMap.isEmpty()) {
+        sentryStore.alterSentryRoleRevokePrivileges(request.getRequestorUserName(),
+            request.getRoleName(), request.getPrivileges(), privilegesUpdateMap);
+      } else {
+        sentryStore.alterSentryRoleRevokePrivileges(request.getRequestorUserName(),
+            request.getRoleName(), request.getPrivileges());
+      }
       response.setStatus(Status.OK());
       notificationHandlerInvoker.alter_sentry_role_revoke_privilege(request,
               response);
-      for (SentryPolicyStorePlugin plugin : sentryPlugins) {
-        plugin.onAlterSentryRoleRevokePrivilege(request);
-      }
     } catch (SentryNoSuchObjectException e) {
       StringBuilder msg = new StringBuilder();
       if (request.getPrivileges().size() > 0) {
@@ -378,12 +398,22 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       validateClientVersion(request.getProtocol_version());
       authorize(request.getRequestorUserName(),
           getRequestorGroups(request.getRequestorUserName()));
-      sentryStore.dropSentryRole(request.getRoleName());
-      response.setStatus(Status.OK());
-      notificationHandlerInvoker.drop_sentry_role(request, response);
+
+      // TODO: now only has SentryPlugin. Once add more SentryPolicyStorePlugins,
+      // TODO: need to differentiate the updates for different Plugins.
+      Preconditions.checkState(sentryPlugins.size() <= 1);
+      Update update = null;
       for (SentryPolicyStorePlugin plugin : sentryPlugins) {
-        plugin.onDropSentryRole(request);
+        update = plugin.onDropSentryRole(request);
+      }
+
+      if (update != null) {
+        sentryStore.dropSentryRole(request.getRoleName(), update);
+      } else {
+        sentryStore.dropSentryRole(request.getRoleName());
       }
+      response.setStatus(Status.OK());
+      notificationHandlerInvoker.drop_sentry_role(request, response);
     } catch (SentryNoSuchObjectException e) {
       String msg = "Role :" + request + " doesn't exist";
       LOGGER.error(msg, e);
@@ -422,15 +452,24 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       validateClientVersion(request.getProtocol_version());
       authorize(request.getRequestorUserName(),
           getRequestorGroups(request.getRequestorUserName()));
-      sentryStore.alterSentryRoleAddGroups(
-          request.getRequestorUserName(), request.getRoleName(),
-          request.getGroups());
-      response.setStatus(Status.OK());
-      notificationHandlerInvoker.alter_sentry_role_add_groups(request,
-              response);
+
+      // TODO: now only has SentryPlugin. Once add more SentryPolicyStorePlugins,
+      // TODO: need to differentiate the updates for different Plugins.
+      Preconditions.checkState(sentryPlugins.size() <= 1);
+      Update update = null;
       for (SentryPolicyStorePlugin plugin : sentryPlugins) {
-        plugin.onAlterSentryRoleAddGroups(request);
+        update = plugin.onAlterSentryRoleAddGroups(request);
       }
+      if (update != null) {
+        sentryStore.alterSentryRoleAddGroups(request.getRequestorUserName(),
+            request.getRoleName(), request.getGroups(), update);
+      } else {
+        sentryStore.alterSentryRoleAddGroups(request.getRequestorUserName(),
+            request.getRoleName(), request.getGroups());
+      }
+      response.setStatus(Status.OK());
+      notificationHandlerInvoker.alter_sentry_role_add_groups(request,
+          response);
     } catch (SentryNoSuchObjectException e) {
       String msg = "Role: " + request + " doesn't exist";
       LOGGER.error(msg, e);
@@ -550,14 +589,25 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       validateClientVersion(request.getProtocol_version());
       authorize(request.getRequestorUserName(),
           getRequestorGroups(request.getRequestorUserName()));
-      sentryStore.alterSentryRoleDeleteGroups(request.getRoleName(),
-              request.getGroups());
-      response.setStatus(Status.OK());
-      notificationHandlerInvoker.alter_sentry_role_delete_groups(request,
-              response);
+
+      // TODO: now only has SentryPlugin. Once add more SentryPolicyStorePlugins,
+      // TODO: need to differentiate the updates for different Plugins.
+      Preconditions.checkState(sentryPlugins.size() <= 1);
+      Update update = null;
       for (SentryPolicyStorePlugin plugin : sentryPlugins) {
-        plugin.onAlterSentryRoleDeleteGroups(request);
+        update = plugin.onAlterSentryRoleDeleteGroups(request);
       }
+
+      if (update != null) {
+        sentryStore.alterSentryRoleDeleteGroups(request.getRoleName(),
+          request.getGroups(), update);
+      } else {
+        sentryStore.alterSentryRoleDeleteGroups(request.getRoleName(),
+          request.getGroups());
+      }
+      response.setStatus(Status.OK());
+      notificationHandlerInvoker.alter_sentry_role_delete_groups(request,
+          response);
     } catch (SentryNoSuchObjectException e) {
       String msg = "Role: " + request + " does not exist.";
       LOGGER.error(msg, e);
@@ -829,9 +879,18 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     try {
       validateClientVersion(request.getProtocol_version());
       authorize(request.getRequestorUserName(), adminGroups);
-      sentryStore.dropPrivilege(request.getAuthorizable());
+
+      // TODO: now only has SentryPlugin. Once add more SentryPolicyStorePlugins,
+      // TODO: need to differentiate the updates for different Plugins.
+      Preconditions.checkState(sentryPlugins.size() <= 1);
+      Update update = null;
       for (SentryPolicyStorePlugin plugin : sentryPlugins) {
-        plugin.onDropSentryPrivilege(request);
+        update = plugin.onDropSentryPrivilege(request);
+      }
+      if (update != null) {
+        sentryStore.dropPrivilege(request.getAuthorizable(), update);
+      } else {
+        sentryStore.dropPrivilege(request.getAuthorizable());
       }
       response.setStatus(Status.OK());
     } catch (SentryAccessDeniedException e) {
@@ -859,10 +918,20 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     try {
       validateClientVersion(request.getProtocol_version());
       authorize(request.getRequestorUserName(), adminGroups);
-      sentryStore.renamePrivilege(request.getOldAuthorizable(),
-          request.getNewAuthorizable());
+
+      // TODO: now only has SentryPlugin. Once add more SentryPolicyStorePlugins,
+      // TODO: need to differentiate the updates for different Plugins.
+      Preconditions.checkState(sentryPlugins.size() <= 1);
+      Update update = null;
       for (SentryPolicyStorePlugin plugin : sentryPlugins) {
-        plugin.onRenameSentryPrivilege(request);
+        update = plugin.onRenameSentryPrivilege(request);
+      }
+      if (update != null) {
+        sentryStore.renamePrivilege(request.getOldAuthorizable(),
+            request.getNewAuthorizable(), update);
+      } else {
+        sentryStore.renamePrivilege(request.getOldAuthorizable(),
+            request.getNewAuthorizable());
       }
       response.setStatus(Status.OK());
     } catch (SentryAccessDeniedException e) {