You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by ka...@apache.org on 2017/08/07 22:11:33 UTC

sentry git commit: SENTRY-1854: HMSFollower should handle notifications even if HDFS sync is disabled. (Na Li Reviewed by Vamsee Yarlagadda, Sergio Pena, kalyan kumar kalvagadda)

Repository: sentry
Updated Branches:
  refs/heads/master 67d64d0d5 -> 1a97f5ab0


SENTRY-1854: HMSFollower should handle notifications even if HDFS sync is disabled. (Na Li Reviewed by Vamsee Yarlagadda, Sergio Pena, kalyan kumar kalvagadda)


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

Branch: refs/heads/master
Commit: 1a97f5ab0c222cdb5bc33461e4f934b530462871
Parents: 67d64d0
Author: Kalyan Kumar Kalvagadda <kk...@cloudera.com>
Authored: Mon Aug 7 17:10:50 2017 -0500
Committer: Kalyan Kumar Kalvagadda <kk...@cloudera.com>
Committed: Mon Aug 7 17:10:50 2017 -0500

----------------------------------------------------------------------
 .../service/persistent/DelegateSentryStore.java |  1 +
 .../db/service/persistent/SentryStore.java      | 57 ++++++++------
 .../db/service/thrift/SentryAdminServlet.java   |  1 +
 .../sentry/service/thrift/HMSFollower.java      |  9 ++-
 .../service/thrift/NotificationProcessor.java   | 66 +++++++++++++---
 .../sentry/service/thrift/SentryService.java    | 14 ++--
 .../service/thrift/SentryServiceUtil.java       | 67 ++++++++++++++--
 .../TestHMSFollowerSentryStoreIntegration.java  | 13 +--
 .../db/service/persistent/TestSentryStore.java  | 74 +++++++++++++++++
 .../persistent/TestSentryStoreImportExport.java |  3 +
 .../service/persistent/TestSentryVersion.java   |  6 ++
 .../sentry/service/thrift/TestHMSFollower.java  | 83 ++++++++++++++++++++
 .../thrift/TestNotificationProcessor.java       | 20 +++++
 13 files changed, 350 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/1a97f5ab/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 4cb46ab..c221c34 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
@@ -40,6 +40,7 @@ import org.apache.sentry.provider.db.service.persistent.TransactionBlock;
 import org.apache.sentry.provider.db.service.thrift.SentryPolicyStoreProcessor;
 import org.apache.sentry.provider.db.service.thrift.TSentryGroup;
 import org.apache.sentry.provider.db.service.thrift.TSentryRole;
+import org.apache.sentry.service.thrift.SentryServiceUtil;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
 
 import com.google.common.annotations.VisibleForTesting;

http://git-wip-us.apache.org/repos/asf/sentry/blob/1a97f5ab/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 f4842a9..d20de26 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
@@ -38,7 +38,6 @@ import javax.jdo.JDOHelper;
 import javax.jdo.PersistenceManager;
 import javax.jdo.PersistenceManagerFactory;
 import javax.jdo.Query;
-import javax.jdo.Transaction;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -172,6 +171,10 @@ public class SentryStore {
   private Configuration conf;
   private final TransactionManager tm;
 
+  // When it is true, execute DeltaTransactionBlock to persist delta changes.
+  // When it is false, do not execute DeltaTransactionBlock
+  private boolean persistUpdateDeltas;
+
   /**
    * counterWait is used to synchronize notifications between Thrift and HMSFollower.
    * Technically it doesn't belong here, but the only thing that connects HMSFollower
@@ -263,6 +266,11 @@ public class SentryStore {
     verifySentryStoreSchema(checkSchemaVersion);
   }
 
+  public void setPersistUpdateDeltas(boolean persistUpdateDeltas) {
+    this.persistUpdateDeltas = persistUpdateDeltas;
+  }
+
+
   public TransactionManager getTransactionManager() {
     return tm;
   }
@@ -730,7 +738,7 @@ public class SentryStore {
       final String roleName, final TSentryPrivilege privilege,
       final Update update) throws Exception {
 
-    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+    execute(update, new TransactionBlock<Object>() {
       public Object execute(PersistenceManager pm) throws Exception {
         pm.setDetachAllOnCommit(false); // No need to detach objects
         String trimmedRoleName = trimAndLower(roleName);
@@ -889,7 +897,7 @@ public class SentryStore {
   private synchronized void alterSentryRoleRevokePrivilege(final String grantorPrincipal,
                                               final String roleName, final TSentryPrivilege tPrivilege,
                                               final Update update) throws Exception {
-    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+    execute(update, new TransactionBlock<Object>() {
       public Object execute(PersistenceManager pm) throws Exception {
         pm.setDetachAllOnCommit(false); // No need to detach objects
         String trimmedRoleName = safeTrimLower(roleName);
@@ -1224,8 +1232,7 @@ public class SentryStore {
    */
   public synchronized void dropSentryRole(final String roleName,
       final Update update) throws Exception {
-
-    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+    execute(update, new TransactionBlock<Object>() {
       public Object execute(PersistenceManager pm) throws Exception {
         pm.setDetachAllOnCommit(false); // No need to detach objects
         dropSentryRoleCore(pm, roleName);
@@ -1303,7 +1310,7 @@ public class SentryStore {
       final String roleName, final Set<TSentryGroup> groupNames,
       final Update update) throws Exception {
 
-    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+    execute(update, new TransactionBlock<Object>() {
       public Object execute(PersistenceManager pm) throws Exception {
         pm.setDetachAllOnCommit(false); // No need to detach objects
         alterSentryRoleAddGroupsCore(pm, roleName, groupNames);
@@ -1452,7 +1459,7 @@ public class SentryStore {
   public synchronized void alterSentryRoleDeleteGroups(final String roleName,
       final Set<TSentryGroup> groupNames, final Update update)
           throws Exception {
-    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+    execute(update, new TransactionBlock<Object>() {
       public Object execute(PersistenceManager pm) throws Exception {
         pm.setDetachAllOnCommit(false); // No need to detach objects
         String trimmedRoleName = trimAndLower(roleName);
@@ -2189,8 +2196,7 @@ public class SentryStore {
    */
   public synchronized void dropPrivilege(final TSentryAuthorizable tAuthorizable,
       final Update update) throws Exception {
-
-    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+    execute(update, new TransactionBlock<Object>() {
       public Object execute(PersistenceManager pm) throws Exception {
         pm.setDetachAllOnCommit(false); // No need to detach objects
 
@@ -2269,7 +2275,7 @@ public class SentryStore {
       final TSentryAuthorizable newTAuthorizable, final Update update)
         throws Exception {
 
-    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+    execute(update, new TransactionBlock<Object>() {
       public Object execute(PersistenceManager pm) throws Exception {
         pm.setDetachAllOnCommit(false); // No need to detach objects
 
@@ -2753,7 +2759,7 @@ public class SentryStore {
    */
   public void addAuthzPathsMapping(final String authzObj, final Collection<String> paths,
       final Update update) throws Exception {
-    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+    execute(update, new TransactionBlock<Object>() {
       public Object execute(PersistenceManager pm) throws Exception {
         pm.setDetachAllOnCommit(false); // No need to detach objects
         addAuthzPathsMappingCore(pm, authzObj, paths);
@@ -2800,7 +2806,7 @@ public class SentryStore {
    */
   public void deleteAuthzPathsMapping(final String authzObj, final Iterable<String> paths,
       final Update update) throws Exception {
-    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+    execute(update, new TransactionBlock<Object>() {
       public Object execute(PersistenceManager pm) throws Exception {
         pm.setDetachAllOnCommit(false); // No need to detach objects
         deleteAuthzPathsMappingCore(pm, authzObj, paths);
@@ -2852,7 +2858,7 @@ public class SentryStore {
    */
   public void deleteAllAuthzPathsMapping(final String authzObj, final Update update)
         throws Exception {
-    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+    execute(update, new TransactionBlock<Object>() {
       public Object execute(PersistenceManager pm) throws Exception {
         pm.setDetachAllOnCommit(false); // No need to detach objects
         deleteAllAuthzPathsMappingCore(pm, authzObj);
@@ -2901,7 +2907,7 @@ public class SentryStore {
    */
   public void renameAuthzPathsMapping(final String oldObj, final String newObj,
       final String oldPath, final String newPath, final Update update) throws Exception {
-    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+    execute(update, new TransactionBlock<Object>() {
       public Object execute(PersistenceManager pm) throws Exception {
         pm.setDetachAllOnCommit(false); // No need to detach objects
         renameAuthzPathsMappingCore(pm, oldObj, newObj, oldPath, newPath);
@@ -2958,7 +2964,7 @@ public class SentryStore {
    */
   public void renameAuthzObj(final String oldObj, final String newObj,
       final Update update) throws Exception {
-    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+    execute(update, new TransactionBlock<Object>() {
       public Object execute(PersistenceManager pm) throws Exception {
         pm.setDetachAllOnCommit(false); // No need to detach objects
         renameAuthzObjCore(pm, oldObj, newObj);
@@ -3041,7 +3047,7 @@ public class SentryStore {
    */
   public void updateAuthzPathsMapping(final String authzObj, final String oldPath,
         final String newPath, final Update update) throws Exception {
-    execute(new DeltaTransactionBlock(update), new TransactionBlock<Object>() {
+    execute(update, new TransactionBlock<Object>() {
       public Object execute(PersistenceManager pm) throws Exception {
         pm.setDetachAllOnCommit(false); // No need to detach objects
         updateAuthzPathsMappingCore(pm, authzObj, oldPath, newPath);
@@ -4080,7 +4086,8 @@ public class SentryStore {
     }
 
     // The first delta change from the DB should match the changeID
-    // If it doesn't, then it means the delta table no longer has entries starting from the requested CHANGE_ID
+    // If it doesn't, then it means the delta table no longer has entries starting from the
+    // requested CHANGE_ID
     if (changes.get(0).getChangeID() != changeID) {
       LOGGER.debug(String.format("Starting delta change from %s is off from the requested id. " +
           "Requested changeID: %s, Missing delta count: %s",
@@ -4101,24 +4108,26 @@ public class SentryStore {
   }
 
   /**
-   * Execute Perm/Path UpdateTransaction and corresponding actual
-   * action transaction, e.g dropSentryRole, in a single transaction.
+   * Execute actual Perm/Path action transaction, e.g dropSentryRole, and persist corresponding
+   * Update in a single transaction if persistUpdateDeltas is true.
    * Note that this method only applies to TransactionBlock that
    * does not have any return value.
    * <p>
    * Failure in any TransactionBlock would cause the whole transaction
    * to fail.
    *
-   * @param deltaTransactionBlock
+   * @param update
    * @param transactionBlock
    * @throws Exception
    */
-  private void execute(DeltaTransactionBlock deltaTransactionBlock,
+  private void execute(Update update,
         TransactionBlock<Object> transactionBlock) throws Exception {
-    List<TransactionBlock<Object>> tbs = Lists.newArrayList();
-    if (deltaTransactionBlock != null) {
-      tbs.add(deltaTransactionBlock);
+    List<TransactionBlock<Object>> tbs = new ArrayList(2);
+
+    if (persistUpdateDeltas) {
+      tbs.add(new DeltaTransactionBlock(update));
     }
+
     tbs.add(transactionBlock);
     tm.executeTransactionBlocksWithRetry(tbs);
   }

http://git-wip-us.apache.org/repos/asf/sentry/blob/1a97f5ab/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryAdminServlet.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryAdminServlet.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryAdminServlet.java
index 8a8bbd3..17dc13f 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryAdminServlet.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryAdminServlet.java
@@ -31,6 +31,7 @@ import java.io.Writer;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
+import org.apache.sentry.service.thrift.SentryServiceUtil;
 
 /**
  * Admin Servlet is only used when SENTRY_WEB_ADMIN_SERVLET_ENABLED is true.

http://git-wip-us.apache.org/repos/asf/sentry/blob/1a97f5ab/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 9e8e0e7..b0a202e 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
@@ -48,6 +48,7 @@ public class HMSFollower implements Runnable, AutoCloseable {
   private final Configuration authzConf;
   private final SentryStore sentryStore;
   private final NotificationProcessor notificationProcessor;
+  private final boolean hdfsSyncEnabled;
 
   private final LeaderStatusMonitor leaderMonitor;
 
@@ -86,6 +87,7 @@ public class HMSFollower implements Runnable, AutoCloseable {
 
     notificationProcessor = new NotificationProcessor(sentryStore, authServerName, authzConf);
     client = new SentryHMSClient(authzConf, hiveConnectionFactory);
+    hdfsSyncEnabled = SentryServiceUtil.isHDFSSyncEnabledNoCache(authzConf); // no cache to test different settings for hdfs sync
   }
 
   @VisibleForTesting
@@ -285,7 +287,12 @@ public class HMSFollower implements Runnable, AutoCloseable {
 
     try {
       LOGGER.debug("Persisting HMS path full snapshot");
-      sentryStore.persistFullPathsImage(snapshotInfo.getPathImage());
+
+      if (hdfsSyncEnabled) {
+        sentryStore.persistFullPathsImage(snapshotInfo.getPathImage());
+      }
+
+      // We need to persist latest notificationID for next poll
       sentryStore.persistLastProcessedNotificationID(snapshotInfo.getId());
     } catch (Exception failure) {
       LOGGER.error("Received exception while persisting HMS path full snapshot ");

http://git-wip-us.apache.org/repos/asf/sentry/blob/1a97f5ab/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java
index f631869..9e28da4 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/NotificationProcessor.java
@@ -78,6 +78,7 @@ final class NotificationProcessor {
   // These variables can be updated even after object is instantiated, for testing purposes.
   private boolean syncStoreOnCreate = false;
   private boolean syncStoreOnDrop = false;
+  private final boolean hdfsSyncEnabled;
 
   /**
    * Configuring notification processor.
@@ -96,6 +97,7 @@ final class NotificationProcessor {
             AUTHZ_SYNC_CREATE_WITH_POLICY_STORE.getDefault()));
     syncStoreOnDrop = Boolean.parseBoolean(conf.get(AUTHZ_SYNC_DROP_WITH_POLICY_STORE.getVar(),
         AUTHZ_SYNC_DROP_WITH_POLICY_STORE.getDefault()));
+    hdfsSyncEnabled = SentryServiceUtil.isHDFSSyncEnabled(conf);
   }
 
   /**
@@ -241,12 +243,19 @@ final class NotificationProcessor {
           StringUtils.defaultIfBlank(location, "null"));
       return false;
     }
-    List<String> locations = Collections.singletonList(location);
-    addPaths(dbName, locations, event.getEventId());
+
     if (syncStoreOnCreate) {
       dropSentryDbPrivileges(dbName, event);
     }
-    return true;
+
+    if (hdfsSyncEnabled) {
+      List<String> locations = Collections.singletonList(location);
+      addPaths(dbName, locations, event.getEventId());
+
+      return true;
+    }
+
+    return false;
   }
 
   /**
@@ -268,9 +277,13 @@ final class NotificationProcessor {
     if (syncStoreOnDrop) {
       dropSentryDbPrivileges(dbName, event);
     }
-    List<String> locations = Collections.singletonList(location);
-    removePaths(dbName, locations, event.getEventId());
-    return true;
+
+    if (hdfsSyncEnabled) {
+      List<String> locations = Collections.singletonList(location);
+      removePaths(dbName, locations, event.getEventId());
+      return true;
+    }
+    return false;
   }
 
   /**
@@ -298,10 +311,15 @@ final class NotificationProcessor {
     if (syncStoreOnCreate) {
       dropSentryTablePrivileges(dbName, tableName, event);
     }
-    String authzObj = SentryServiceUtil.getAuthzObj(dbName, tableName);
-    List<String> locations = Collections.singletonList(location);
-    addPaths(authzObj, locations, event.getEventId());
-    return true;
+
+    if (hdfsSyncEnabled) {
+      String authzObj = SentryServiceUtil.getAuthzObj(dbName, tableName);
+      List<String> locations = Collections.singletonList(location);
+      addPaths(authzObj, locations, event.getEventId());
+      return true;
+    }
+
+    return false;
   }
 
   /**
@@ -327,9 +345,14 @@ final class NotificationProcessor {
     if (syncStoreOnDrop) {
       dropSentryTablePrivileges(dbName, tableName, event);
     }
-    String authzObj = SentryServiceUtil.getAuthzObj(dbName, tableName);
-    removeAllPaths(authzObj, event.getEventId());
-    return true;
+
+    if (hdfsSyncEnabled) {
+      String authzObj = SentryServiceUtil.getAuthzObj(dbName, tableName);
+      removeAllPaths(authzObj, event.getEventId());
+      return true;
+    }
+
+    return false;
   }
 
   /**
@@ -340,6 +363,11 @@ final class NotificationProcessor {
    * @throws Exception if encounters errors while persisting the path change
    */
   private boolean processAlterTable(NotificationEvent event) throws Exception {
+
+    if (!hdfsSyncEnabled) {
+      return false;
+    }
+
     SentryJSONAlterTableMessage alterTableMessage =
         deserializer.getAlterTableMessage(event.getMessage());
     String oldDbName = alterTableMessage.getDB();
@@ -404,6 +432,10 @@ final class NotificationProcessor {
    */
   private boolean processAddPartition(NotificationEvent event)
       throws Exception {
+    if (!hdfsSyncEnabled) {
+      return false;
+    }
+
     SentryJSONAddPartitionMessage addPartitionMessage =
         deserializer.getAddPartitionMessage(event.getMessage());
     String dbName = addPartitionMessage.getDB();
@@ -431,6 +463,10 @@ final class NotificationProcessor {
    */
   private boolean processDropPartition(NotificationEvent event)
       throws Exception {
+    if (!hdfsSyncEnabled) {
+      return false;
+    }
+
     SentryJSONDropPartitionMessage dropPartitionMessage =
         deserializer.getDropPartitionMessage(event.getMessage());
     String dbName = dropPartitionMessage.getDB();
@@ -457,6 +493,10 @@ final class NotificationProcessor {
    * @throws Exception if encounters errors while persisting the path change
    */
   private boolean processAlterPartition(NotificationEvent event) throws Exception {
+    if (!hdfsSyncEnabled) {
+      return false;
+    }
+
     SentryJSONAlterPartitionMessage alterPartitionMessage =
         deserializer.getAlterPartitionMessage(event.getMessage());
     String dbName = alterPartitionMessage.getDB();

http://git-wip-us.apache.org/repos/asf/sentry/blob/1a97f5ab/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
index 10d55dc..d44abff 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryService.java
@@ -113,7 +113,6 @@ public class SentryService implements Callable, SigUtils.SigListener {
   private final SentryStore sentryStore;
   private ScheduledExecutorService sentryStoreCleanService;
   private final LeaderStatusMonitor leaderMonitor;
-  private final boolean hdfsSyncEnabled;
 
   public SentryService(Configuration conf) throws Exception {
     this.conf = conf;
@@ -163,11 +162,10 @@ public class SentryService implements Callable, SigUtils.SigListener {
         .build();
     serviceExecutor = Executors.newSingleThreadExecutor(sentryServiceThreadFactory);
     this.sentryStore = new SentryStore(conf);
+    sentryStore.setPersistUpdateDeltas(SentryServiceUtil.isHDFSSyncEnabled(conf));
     this.leaderMonitor = LeaderStatusMonitor.getLeaderStatusMonitor(conf);
     webServerPort = conf.getInt(ServerConfig.SENTRY_WEB_PORT, ServerConfig.SENTRY_WEB_PORT_DEFAULT);
 
-    hdfsSyncEnabled = SentryServiceUtil.isHDFSSyncEnabled(conf);
-
     status = Status.NOT_STARTED;
 
     // Enable signal handler for HA leader/follower status if configured
@@ -276,8 +274,10 @@ public class SentryService implements Callable, SigUtils.SigListener {
   }
 
   private void startHMSFollower(Configuration conf) throws Exception {
-    if (!hdfsSyncEnabled) {
-      LOGGER.info("HMS follower is not started because HDFS sync is disabled.");
+    boolean syncPolicyStore = SentryServiceUtil.isSyncPolicyStoreEnabled(conf);
+
+    if ((!SentryServiceUtil.isHDFSSyncEnabled(conf)) && (!syncPolicyStore)) {
+      LOGGER.info("HMS follower is not started because HDFS sync is disabled and perm sync is disabled");
       return;
     }
 
@@ -315,10 +315,6 @@ public class SentryService implements Callable, SigUtils.SigListener {
   }
 
   private void stopHMSFollower(Configuration conf) {
-    if (!hdfsSyncEnabled) {
-      return;
-    }
-
     if ((hmsFollowerExecutor == null) || (hmsFollower == null)) {
         Preconditions.checkState(hmsFollower == null);
         Preconditions.checkState(hmsFollowerExecutor == null);

http://git-wip-us.apache.org/repos/asf/sentry/blob/1a97f5ab/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryServiceUtil.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryServiceUtil.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryServiceUtil.java
index 5826766..3488e11 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryServiceUtil.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryServiceUtil.java
@@ -29,6 +29,10 @@ import com.google.common.base.Preconditions;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS;
+import static org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars.AUTHZ_SYNC_ALTER_WITH_POLICY_STORE;
+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 org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.sentry.core.common.exception.SentryInvalidInputException;
 import org.apache.sentry.core.common.utils.SentryConstants;
@@ -45,6 +49,9 @@ import org.slf4j.Logger;
 
 public final class SentryServiceUtil {
 
+  private static boolean firstCallHDFSSyncEnabled = true;
+  private static boolean hdfsSyncEnabled = false;
+
   // parse the privilege in String and get the TSentryPrivilege as result
   public static TSentryPrivilege convertToTSentryPrivilege(String privilegeStr) {
     TSentryPrivilege tSentryPrivilege = new TSentryPrivilege();
@@ -200,21 +207,71 @@ public final class SentryServiceUtil {
   }
 
   /**
-   * Checks if Sentry is configured with HDFS sync enabled.
+   * Check if Sentry is configured with HDFS sync enabled. Cache the result
    *
    * @param conf The Configuration object where HDFS sync configurations are set.
    * @return True if enabled; False otherwise.
    */
-  static boolean isHDFSSyncEnabled(Configuration conf) {
+  public static boolean isHDFSSyncEnabled(Configuration conf) {
+    if (firstCallHDFSSyncEnabled) {
+      List<String> processorFactories =
+          Arrays.asList(conf.get(ServiceConstants.ServerConfig.PROCESSOR_FACTORIES, "").split(","));
+
+      List<String> policyStorePlugins =
+          Arrays.asList(
+              conf.get(ServiceConstants.ServerConfig.SENTRY_POLICY_STORE_PLUGINS, "").split(","));
+
+      hdfsSyncEnabled =
+          processorFactories.contains("org.apache.sentry.hdfs.SentryHDFSServiceProcessorFactory")
+              && policyStorePlugins.contains("org.apache.sentry.hdfs.SentryPlugin");
+      firstCallHDFSSyncEnabled = false;
+    }
+
+    return hdfsSyncEnabled;
+  }
+
+    /**
+     * Check if Sentry is configured with HDFS sync enabled without caching the result
+     *
+     * @param conf The Configuration object where HDFS sync configurations are set.
+     * @return True if enabled; False otherwise.
+     */
+  public static boolean isHDFSSyncEnabledNoCache(Configuration conf) {
+
     List<String> processorFactories =
         Arrays.asList(conf.get(ServiceConstants.ServerConfig.PROCESSOR_FACTORIES, "").split(","));
 
     List<String> policyStorePlugins =
-        Arrays.asList(conf.get(ServiceConstants.ServerConfig.SENTRY_POLICY_STORE_PLUGINS, "").split(","));
+        Arrays.asList(
+            conf.get(ServiceConstants.ServerConfig.SENTRY_POLICY_STORE_PLUGINS, "").split(","));
+
+    hdfsSyncEnabled =
+        processorFactories.contains("org.apache.sentry.hdfs.SentryHDFSServiceProcessorFactory")
+            && policyStorePlugins.contains("org.apache.sentry.hdfs.SentryPlugin");
+
+
+    return hdfsSyncEnabled;
+  }
+
+  /**
+   * Check if Sentry is configured with policy store sync enabled
+   * @param conf
+   * @return True if enabled; False otherwise
+   */
+  public static boolean isSyncPolicyStoreEnabled(Configuration conf) {
+    boolean syncStoreOnCreate;
+    boolean syncStoreOnDrop;
+    boolean syncStoreOnAlter;
 
+    syncStoreOnCreate  = Boolean
+        .parseBoolean(conf.get(AUTHZ_SYNC_CREATE_WITH_POLICY_STORE.getVar(),
+            AUTHZ_SYNC_CREATE_WITH_POLICY_STORE.getDefault()));
+    syncStoreOnDrop = Boolean.parseBoolean(conf.get(AUTHZ_SYNC_DROP_WITH_POLICY_STORE.getVar(),
+        AUTHZ_SYNC_DROP_WITH_POLICY_STORE.getDefault()));
+    syncStoreOnAlter = Boolean.parseBoolean(conf.get(AUTHZ_SYNC_ALTER_WITH_POLICY_STORE.getVar(),
+        AUTHZ_SYNC_ALTER_WITH_POLICY_STORE.getDefault()));
 
-    return processorFactories.contains("org.apache.sentry.hdfs.SentryHDFSServiceProcessorFactory")
-        && policyStorePlugins.contains("org.apache.sentry.hdfs.SentryPlugin");
+    return syncStoreOnCreate || syncStoreOnDrop || syncStoreOnAlter;
   }
 
   static String getHiveMetastoreURI() {

http://git-wip-us.apache.org/repos/asf/sentry/blob/1a97f5ab/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestHMSFollowerSentryStoreIntegration.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestHMSFollowerSentryStoreIntegration.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestHMSFollowerSentryStoreIntegration.java
index 82f600b..4ee1597 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestHMSFollowerSentryStoreIntegration.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestHMSFollowerSentryStoreIntegration.java
@@ -82,7 +82,6 @@ public class TestHMSFollowerSentryStoreIntegration {
     // THis should be a UserGroupInformation provider
     CredentialProvider provider = CredentialProviderFactory.getProviders(conf).get(0);
 
-
     // The user credentials are stored as a static variable by UserGrouoInformation provider.
     // We need to only set the password the first time, an attempt to set it for the second
     // time fails with an exception.
@@ -109,6 +108,7 @@ public class TestHMSFollowerSentryStoreIntegration {
   @Before
   public void before() throws Exception {
     sentryStore = new SentryStore(conf);
+    sentryStore.setPersistUpdateDeltas(true);
     policyFile = new PolicyFile();
     String adminUser = "g1";
     addGroupsToUser(adminUser, adminGroups);
@@ -155,17 +155,6 @@ public class TestHMSFollowerSentryStoreIntegration {
         sentryStore.getMSentryRoleByName(roleName).getRoleName());
   }
 
-  /**
-   * Create a role with the given name and verify that it is created
-   * @param roleName
-   * @throws Exception
-   */
-  private void createRole(String roleName) throws Exception {
-    checkRoleDoesNotExist(roleName);
-    sentryStore.createSentryRole(roleName);
-    checkRoleExists(roleName);
-  }
-
   private TSentryAuthorizable toTSentryAuthorizable(
       TSentryPrivilege tSentryPrivilege) {
     TSentryAuthorizable tSentryAuthorizable = new TSentryAuthorizable();

http://git-wip-us.apache.org/repos/asf/sentry/blob/1a97f5ab/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 a8ebf7c..859a9fe 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
@@ -59,6 +59,8 @@ import org.apache.sentry.provider.db.service.thrift.TSentryGroup;
 import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
 import org.apache.sentry.provider.db.service.thrift.TSentryRole;
 import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.service.thrift.SentryServiceUtil;
+import org.apache.sentry.service.thrift.ServiceConstants;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -96,6 +98,10 @@ public class TestSentryStore extends org.junit.Assert {
     final String ourUrl = UserProvider.SCHEME_NAME + ":///";
     conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, ourUrl);
 
+    // enable HDFS sync, so perm and path changes will be saved into DB
+    conf.set(ServiceConstants.ServerConfig.PROCESSOR_FACTORIES, "org.apache.sentry.hdfs.SentryHDFSServiceProcessorFactory");
+    conf.set(ServiceConstants.ServerConfig.SENTRY_POLICY_STORE_PLUGINS, "org.apache.sentry.hdfs.SentryPlugin");
+
     // THis should be a UserGroupInformation provider
     CredentialProvider provider = CredentialProviderFactory.getProviders(conf).get(0);
 
@@ -120,7 +126,9 @@ public class TestSentryStore extends org.junit.Assert {
     conf.set(ServerConfig.SENTRY_STORE_GROUP_MAPPING_RESOURCE,
         policyFilePath.getPath());
     conf.setInt(ServerConfig.SENTRY_STORE_TRANSACTION_RETRY, 10);
+    boolean hdfsSyncEnabled = SentryServiceUtil.isHDFSSyncEnabled(conf);
     sentryStore = new SentryStore(conf);
+    sentryStore.setPersistUpdateDeltas(hdfsSyncEnabled);
   }
 
   @Before
@@ -3244,4 +3252,70 @@ public class TestSentryStore extends org.junit.Assert {
     sentryStore.clearAllTables();
     assertEquals(sentryStore.isAuthzPathsMappingEmpty(), true);
   }
+
+  @Test
+  public void testAddDeleteAuthzPathsMappingNoDeltaSavedWithoutHDFSSync() throws Exception {
+
+    // disable HDFS
+    conf.set(ServiceConstants.ServerConfig.PROCESSOR_FACTORIES, "");
+    conf.set(ServiceConstants.ServerConfig.SENTRY_POLICY_STORE_PLUGINS, "");
+    SentryStore localSentryStore = new SentryStore(conf);
+
+    // Persist an empty image so that we can add paths to it.
+    localSentryStore.persistFullPathsImage(new HashMap<String, Set<String>>());
+
+    // Add "db1.table1" authzObj
+    Long lastNotificationId = sentryStore.getLastProcessedNotificationID();
+    PathsUpdate addUpdate = new PathsUpdate(1, false);
+    addUpdate.newPathChange("db1.table").
+        addToAddPaths(Arrays.asList("db1", "tbl1"));
+    addUpdate.newPathChange("db1.table").
+        addToAddPaths(Arrays.asList("db1", "tbl2"));
+
+    localSentryStore.addAuthzPathsMapping("db1.table",
+        Sets.newHashSet("db1/tbl1", "db1/tbl2"), addUpdate);
+    PathsImage pathsImage = localSentryStore.retrieveFullPathsImage();
+    Map<String, Set<String>> pathImage = pathsImage.getPathImage();
+    assertEquals(1, pathImage.size());
+    assertEquals(2, pathImage.get("db1.table").size());
+    assertEquals(2, localSentryStore.getMPaths().size());
+
+    // Query the persisted path change and ensure it is not saved
+    long lastChangeID = localSentryStore.getLastProcessedPathChangeID();
+    assertEquals(0, lastChangeID);
+
+    // Delete path 'db1.db/tbl1' from "db1.table1" authzObj.
+    PathsUpdate delUpdate = new PathsUpdate(2, false);
+    delUpdate.newPathChange("db1.table")
+        .addToDelPaths(Arrays.asList("db1", "tbl1"));
+    localSentryStore.deleteAuthzPathsMapping("db1.table", Sets.newHashSet("db1/tbl1"), delUpdate);
+    pathImage = localSentryStore.retrieveFullPathsImage().getPathImage();
+    assertEquals(1, pathImage.size());
+    assertEquals(1, pathImage.get("db1.table").size());
+    assertEquals(1, localSentryStore.getMPaths().size());
+
+    // Query the persisted path change and ensure it is not saved
+    lastChangeID = localSentryStore.getLastProcessedPathChangeID();
+    assertEquals(0, lastChangeID);
+
+    // Delete "db1.table" authzObj from the authzObj -> [Paths] mapping.
+    PathsUpdate delAllupdate = new PathsUpdate(3, false);
+    delAllupdate.newPathChange("db1.table")
+        .addToDelPaths(Lists.newArrayList(PathsUpdate.ALL_PATHS));
+    localSentryStore.deleteAllAuthzPathsMapping("db1.table", delAllupdate);
+    pathImage = localSentryStore.retrieveFullPathsImage().getPathImage();
+    assertEquals(0, pathImage.size());
+    assertEquals(0, localSentryStore.getMPaths().size());
+
+    // Query the persisted path change and ensure it is not saved
+    lastChangeID = localSentryStore.getLastProcessedPathChangeID();
+    assertEquals(0, lastChangeID);
+
+    lastNotificationId = localSentryStore.getLastProcessedNotificationID();
+    assertEquals(0, lastNotificationId.longValue());
+
+    // enable HDFS for other tests
+    conf.set(ServiceConstants.ServerConfig.PROCESSOR_FACTORIES, "org.apache.sentry.hdfs.SentryHDFSServiceProcessorFactory");
+    conf.set(ServiceConstants.ServerConfig.SENTRY_POLICY_STORE_PLUGINS, "org.apache.sentry.hdfs.SentryPlugin");
+  }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/1a97f5ab/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStoreImportExport.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStoreImportExport.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStoreImportExport.java
index 1c3a4f2..b085ac3 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStoreImportExport.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryStoreImportExport.java
@@ -37,6 +37,7 @@ import org.apache.sentry.provider.db.service.thrift.TSentryGrantOption;
 import org.apache.sentry.provider.db.service.thrift.TSentryMappingData;
 import org.apache.sentry.provider.db.service.thrift.TSentryPrivilege;
 import org.apache.sentry.provider.file.PolicyFile;
+import org.apache.sentry.service.thrift.SentryServiceUtil;
 import org.apache.sentry.service.thrift.ServiceConstants.PrivilegeScope;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
 import org.junit.After;
@@ -79,7 +80,9 @@ public class TestSentryStoreImportExport {
     policyFilePath = new File(dataDir, "local_policy_file.ini");
     conf.set(ServerConfig.SENTRY_STORE_GROUP_MAPPING_RESOURCE, policyFilePath.getPath());
     policyFile = new PolicyFile();
+    boolean hdfsSyncEnabled = SentryServiceUtil.isHDFSSyncEnabled(conf);
     sentryStore = new SentryStore(conf);
+    sentryStore.setPersistUpdateDeltas(hdfsSyncEnabled);
 
     String adminUser = "g1";
     addGroupsToUser(adminUser, adminGroups);

http://git-wip-us.apache.org/repos/asf/sentry/blob/1a97f5ab/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryVersion.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryVersion.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryVersion.java
index a8e8a03..2b7dcde 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryVersion.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/provider/db/service/persistent/TestSentryVersion.java
@@ -24,6 +24,7 @@ import java.io.File;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.sentry.core.common.exception.SentryNoSuchObjectException;
+import org.apache.sentry.service.thrift.SentryServiceUtil;
 import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
 import org.junit.Before;
 import org.junit.Test;
@@ -53,10 +54,13 @@ public class TestSentryVersion {
   @Test
   public void testVerifySentryVersionCheck() throws Exception {
     conf.set(ServerConfig.SENTRY_VERIFY_SCHEM_VERSION, "false");
+    boolean hdfsSyncEnabled = SentryServiceUtil.isHDFSSyncEnabled(conf);
     SentryStore sentryStore = new SentryStore(conf);
+    sentryStore.setPersistUpdateDeltas(hdfsSyncEnabled);
     sentryStore.stop();
     conf.set(ServerConfig.SENTRY_VERIFY_SCHEM_VERSION, "true");
     sentryStore = new SentryStore(conf);
+    sentryStore.setPersistUpdateDeltas(hdfsSyncEnabled);
   }
 
   /**
@@ -77,7 +81,9 @@ public class TestSentryVersion {
   @Test
   public void testSentryImplicitVersion() throws Exception {
     conf.set(ServerConfig.SENTRY_VERIFY_SCHEM_VERSION, "false");
+    boolean hdfsSyncEnabled = SentryServiceUtil.isHDFSSyncEnabled(conf);
     SentryStore sentryStore = new SentryStore(conf);
+    sentryStore.setPersistUpdateDeltas(hdfsSyncEnabled);
     assertEquals(SentryStoreSchemaInfo.getSentryVersion(),
         sentryStore.getSentryVersion());
   }

http://git-wip-us.apache.org/repos/asf/sentry/blob/1a97f5ab/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 23f8de8..6820a9b 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
@@ -66,6 +66,10 @@ public class TestHMSFollower {
     hiveConnectionFactory = new HiveSimpleConnectionFactory(configuration, new HiveConf());
     hiveConnectionFactory.init();
     configuration.set("sentry.hive.sync.create", "true");
+
+    // enable HDFS sync, so perm and path changes will be saved into DB
+    configuration.set(ServiceConstants.ServerConfig.PROCESSOR_FACTORIES, "org.apache.sentry.hdfs.SentryHDFSServiceProcessorFactory");
+    configuration.set(ServiceConstants.ServerConfig.SENTRY_POLICY_STORE_PLUGINS, "org.apache.sentry.hdfs.SentryPlugin");
   }
 
   @Test
@@ -606,6 +610,10 @@ public class TestHMSFollower {
         Mockito.anyCollection(), Mockito.any(Updateable.Update.class));
 
     Configuration configuration = new Configuration();
+    // enable HDFS sync, so perm and path changes will be saved into DB
+    configuration.set(ServiceConstants.ServerConfig.PROCESSOR_FACTORIES, "org.apache.sentry.hdfs.SentryHDFSServiceProcessorFactory");
+    configuration.set(ServiceConstants.ServerConfig.SENTRY_POLICY_STORE_PLUGINS, "org.apache.sentry.hdfs.SentryPlugin");
+
     HMSFollower hmsFollower = new HMSFollower(configuration, sentryStore, null,
         hiveConnectionFactory, hiveInstance);
 
@@ -739,4 +747,79 @@ public class TestHMSFollower {
     verify(sentryStore, times(1)).addAuthzPathsMapping(Mockito.anyString(), Mockito.anyCollection(),
         Mockito.any(Updateable.Update.class));
   }
+
+  @Test
+  public void testNoHdfsNoPersistAFullSnapshot() throws Exception {
+    /*
+     * TEST CASE
+     *
+     * Simulates (by using mocks) that Sentry has not processed any notifications, so this
+     * should trigger a new full HMS snapshot request with the eventId = 1
+     */
+
+    final long SENTRY_PROCESSED_EVENT_ID = SentryStore.EMPTY_NOTIFICATION_ID;
+    final long HMS_PROCESSED_EVENT_ID = 1L;
+
+    // Mock that returns a full snapshot
+    Map<String, Set<String>> snapshotObjects = new HashMap<>();
+    snapshotObjects.put("db", Sets.newHashSet("/db"));
+    snapshotObjects.put("db.table", Sets.newHashSet("/db/table"));
+    PathsImage fullSnapshot = new PathsImage(snapshotObjects, HMS_PROCESSED_EVENT_ID, 1);
+
+    SentryHMSClient sentryHmsClient = Mockito.mock(SentryHMSClient.class);
+    Mockito.when(sentryHmsClient.getFullSnapshot()).thenReturn(fullSnapshot);
+    Mockito.when(sentryHmsClient.getCurrentNotificationId()).thenReturn(fullSnapshot.getId());
+
+    Configuration configuration = new Configuration();
+    HMSFollower hmsFollower = new HMSFollower(configuration, sentryStore, null,
+        hiveConnectionFactory, hiveInstance);
+    hmsFollower.setSentryHmsClient(sentryHmsClient);
+
+    // 1st run should get a full snapshot because AuthzPathsMapping is empty
+    Mockito.when(sentryStore.getLastProcessedNotificationID()).thenReturn(SENTRY_PROCESSED_EVENT_ID);
+    Mockito.when(sentryStore.isAuthzPathsMappingEmpty()).thenReturn(true);
+    hmsFollower.run();
+    Mockito.verify(sentryStore, times(0)).persistFullPathsImage(fullSnapshot.getPathImage());
+    Mockito.verify(sentryStore, times(1)).persistLastProcessedNotificationID(fullSnapshot.getId());
+  }
+
+  @Test
+  public void testNoHdfsSyncAlterTableNotPersisted() throws Exception {
+    String dbName = "db1";
+    String tableName = "table1";
+    String newDbName = "db1";
+    String newTableName = "table2";
+
+    // Create notification events
+    StorageDescriptor sd = new StorageDescriptor();
+    sd.setLocation("hdfs:///db1.db/table1");
+    NotificationEvent notificationEvent = new NotificationEvent(1, 0,
+        HCatEventMessage.EventType.ALTER_TABLE.toString(),
+        messageFactory.buildAlterTableMessage(
+            new Table(tableName, dbName, null, 0, 0, 0, sd, null, null, null, null, null),
+            new Table(newTableName, newDbName, null, 0, 0, 0, sd, null, null, null, null, null))
+            .toString());
+    notificationEvent.setDbName(newDbName);
+    notificationEvent.setTableName(newTableName);
+    List<NotificationEvent> events = new ArrayList<>();
+    events.add(notificationEvent);
+
+    Configuration configuration = new Configuration();
+    HMSFollower hmsFollower = new HMSFollower(configuration, sentryStore, null,
+        hiveConnectionFactory, hiveInstance);
+    hmsFollower.processNotifications(events);
+
+    TSentryAuthorizable authorizable = new TSentryAuthorizable(hiveInstance);
+    authorizable.setServer(hiveInstance);
+    authorizable.setDb(dbName);
+    authorizable.setTable(tableName);
+
+    TSentryAuthorizable newAuthorizable = new TSentryAuthorizable(hiveInstance);
+    authorizable.setServer(hiveInstance);
+    newAuthorizable.setDb(newDbName);
+    newAuthorizable.setTable(newTableName);
+
+    verify(sentryStore, times(0)).renamePrivilege(authorizable, newAuthorizable,
+        NotificationProcessor.getPermUpdatableOnRename(authorizable, newAuthorizable));
+  }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/1a97f5ab/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestNotificationProcessor.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestNotificationProcessor.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestNotificationProcessor.java
index c6c9448..0fd4ff0 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestNotificationProcessor.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestNotificationProcessor.java
@@ -54,6 +54,10 @@ public class TestNotificationProcessor {
   public static void setup() {
     conf.set("sentry.hive.sync.create", "true");
     conf.set("sentry.hive.sync.drop", "true");
+
+    // enable HDFS sync, so perm and path changes will be saved into DB
+    conf.set(ServiceConstants.ServerConfig.PROCESSOR_FACTORIES, "org.apache.sentry.hdfs.SentryHDFSServiceProcessorFactory");
+    conf.set(ServiceConstants.ServerConfig.SENTRY_POLICY_STORE_PLUGINS, "org.apache.sentry.hdfs.SentryPlugin");
   }
 
   @After
@@ -253,6 +257,10 @@ public class TestNotificationProcessor {
     String tableName = "table1";
 
     Configuration authConf = new Configuration();
+    // enable HDFS sync, so perm and path changes will be saved into DB
+    authConf.set(ServiceConstants.ServerConfig.PROCESSOR_FACTORIES, "org.apache.sentry.hdfs.SentryHDFSServiceProcessorFactory");
+    authConf.set(ServiceConstants.ServerConfig.SENTRY_POLICY_STORE_PLUGINS, "org.apache.sentry.hdfs.SentryPlugin");
+
     notificationProcessor = new NotificationProcessor(sentryStore,
         hiveInstance, authConf);
 
@@ -291,6 +299,10 @@ public class TestNotificationProcessor {
     String newTableName = "table2";
 
     Configuration authConf = new Configuration();
+    // enable HDFS sync, so perm and path changes will be saved into DB
+    authConf.set(ServiceConstants.ServerConfig.PROCESSOR_FACTORIES, "org.apache.sentry.hdfs.SentryHDFSServiceProcessorFactory");
+    authConf.set(ServiceConstants.ServerConfig.SENTRY_POLICY_STORE_PLUGINS, "org.apache.sentry.hdfs.SentryPlugin");
+
     notificationProcessor = new NotificationProcessor(sentryStore,
         hiveInstance, authConf);
 
@@ -338,6 +350,10 @@ public class TestNotificationProcessor {
     String newTableName = "table2";
 
     Configuration authConf = new Configuration();
+    // enable HDFS sync, so perm and path changes will be saved into DB
+    authConf.set(ServiceConstants.ServerConfig.PROCESSOR_FACTORIES, "org.apache.sentry.hdfs.SentryHDFSServiceProcessorFactory");
+    authConf.set(ServiceConstants.ServerConfig.SENTRY_POLICY_STORE_PLUGINS, "org.apache.sentry.hdfs.SentryPlugin");
+
     notificationProcessor = new NotificationProcessor(sentryStore,
         hiveInstance, authConf);
 
@@ -393,6 +409,10 @@ public class TestNotificationProcessor {
         Mockito.anyCollection(), Mockito.any(Updateable.Update.class));
 
     Configuration authConf = new Configuration();
+    // enable HDFS sync, so perm and path changes will be saved into DB
+    authConf.set(ServiceConstants.ServerConfig.PROCESSOR_FACTORIES, "org.apache.sentry.hdfs.SentryHDFSServiceProcessorFactory");
+    authConf.set(ServiceConstants.ServerConfig.SENTRY_POLICY_STORE_PLUGINS, "org.apache.sentry.hdfs.SentryPlugin");
+
     notificationProcessor = new NotificationProcessor(sentryStore,
         hiveInstance, authConf);