You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by ak...@apache.org on 2017/09/02 01:37:00 UTC

sentry git commit: SENTRY-1909: Improvements for memory usage when full path snapshot is sent from Sentry to NN (Alex Kolbasov, reviewed by Misha Dmitriev, Sergio Pena and Vamsee Yarlagadda)

Repository: sentry
Updated Branches:
  refs/heads/master bfb07b4a5 -> 7d18d1bf7


SENTRY-1909: Improvements for memory usage when full path snapshot is sent from Sentry to NN (Alex Kolbasov, reviewed by Misha Dmitriev, Sergio Pena and Vamsee Yarlagadda)


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

Branch: refs/heads/master
Commit: 7d18d1bf79efd6f27319b99de9f7635d98f6846b
Parents: bfb07b4
Author: Alexander Kolbasov <ak...@gmail.com>
Authored: Fri Sep 1 18:35:44 2017 -0700
Committer: Alexander Kolbasov <ak...@gmail.com>
Committed: Fri Sep 1 18:35:44 2017 -0700

----------------------------------------------------------------------
 .../apache/sentry/hdfs/PathImageRetriever.java  |   6 +-
 .../apache/sentry/hdfs/TestImageRetriever.java  |  10 +-
 .../hdfs/TestSentryHDFSServiceProcessor.java    |   4 +-
 .../db/service/model/MAuthzPathsMapping.java    |  18 +--
 .../db/service/persistent/PathsImage.java       |   8 +-
 .../db/service/persistent/SentryStore.java      |  11 +-
 .../service/thrift/FullUpdateInitializer.java   |   6 +-
 .../service/thrift/FullUpdateModifier.java      |  53 ++++----
 .../sentry/service/thrift/SentryHMSClient.java  |  33 +++--
 .../db/service/persistent/TestSentryStore.java  | 126 ++++++++++---------
 .../thrift/TestFullUpdateInitializer.java       |   6 +-
 .../service/thrift/TestFullUpdateModifier.java  |  35 +++---
 .../sentry/service/thrift/TestHMSFollower.java  |  11 +-
 13 files changed, 164 insertions(+), 163 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sentry/blob/7d18d1bf/sentry-hdfs/sentry-hdfs-service/src/main/java/org/apache/sentry/hdfs/PathImageRetriever.java
----------------------------------------------------------------------
diff --git a/sentry-hdfs/sentry-hdfs-service/src/main/java/org/apache/sentry/hdfs/PathImageRetriever.java b/sentry-hdfs/sentry-hdfs-service/src/main/java/org/apache/sentry/hdfs/PathImageRetriever.java
index 898c7be..69e43fd 100644
--- a/sentry-hdfs/sentry-hdfs-service/src/main/java/org/apache/sentry/hdfs/PathImageRetriever.java
+++ b/sentry-hdfs/sentry-hdfs-service/src/main/java/org/apache/sentry/hdfs/PathImageRetriever.java
@@ -25,9 +25,9 @@ import org.apache.sentry.provider.db.service.persistent.SentryStore;
 
 import javax.annotation.concurrent.ThreadSafe;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 /**
@@ -58,14 +58,14 @@ class PathImageRetriever implements ImageRetriever<PathsUpdate> {
       PathsImage pathsImage = sentryStore.retrieveFullPathsImage();
       long curImgNum = pathsImage.getCurImgNum();
       long curSeqNum = pathsImage.getId();
-      Map<String, Set<String>> pathImage = pathsImage.getPathImage();
+      Map<String, Collection<String>> pathImage = pathsImage.getPathImage();
 
       // Translates the complete Hive paths snapshot into a PathsUpdate.
       // Adds all <hiveObj, paths> mapping to be included in this paths update.
       // And label it with the latest delta change sequence number for consumer
       // to be aware of the next delta change it should continue with.
       PathsUpdate pathsUpdate = new PathsUpdate(curSeqNum, curImgNum, true);
-      for (Map.Entry<String, Set<String>> pathEnt : pathImage.entrySet()) {
+      for (Map.Entry<String, Collection<String>> pathEnt : pathImage.entrySet()) {
         TPathChanges pathChange = pathsUpdate.newPathChange(pathEnt.getKey());
 
         for (String path : pathEnt.getValue()) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/7d18d1bf/sentry-hdfs/sentry-hdfs-service/src/test/java/org/apache/sentry/hdfs/TestImageRetriever.java
----------------------------------------------------------------------
diff --git a/sentry-hdfs/sentry-hdfs-service/src/test/java/org/apache/sentry/hdfs/TestImageRetriever.java b/sentry-hdfs/sentry-hdfs-service/src/test/java/org/apache/sentry/hdfs/TestImageRetriever.java
index 20b3e10..478ccf9 100644
--- a/sentry-hdfs/sentry-hdfs-service/src/test/java/org/apache/sentry/hdfs/TestImageRetriever.java
+++ b/sentry-hdfs/sentry-hdfs-service/src/test/java/org/apache/sentry/hdfs/TestImageRetriever.java
@@ -26,10 +26,10 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -45,7 +45,7 @@ public class TestImageRetriever {
   @Test
   public void testEmptyPathUpdatesRetrievedWhenNoImagesArePersisted() throws Exception {
     Mockito.when(sentryStoreMock.retrieveFullPathsImage())
-        .thenReturn(new PathsImage(new HashMap<String, Set<String>>(), 0, 0));
+        .thenReturn(new PathsImage(new HashMap<String, Collection<String>>(), 0, 0));
 
     PathImageRetriever imageRetriever = new PathImageRetriever(sentryStoreMock);
     PathsUpdate pathsUpdate = imageRetriever.retrieveFullImage();
@@ -60,7 +60,7 @@ public class TestImageRetriever {
     PathImageRetriever imageRetriever;
     PathsUpdate pathsUpdate;
 
-    Map<String, Set<String>> fullPathsImage = new HashMap<>();
+    Map<String, Collection<String>> fullPathsImage = new HashMap<>();
     fullPathsImage.put("db1", Sets.newHashSet("/user/db1"));
     fullPathsImage.put("db1.table1", Sets.newHashSet("/user/db1/table1"));
 
@@ -76,7 +76,7 @@ public class TestImageRetriever {
     assertTrue(comparePaths(fullPathsImage, pathsUpdate.getPathChanges()));
   }
 
-  private boolean comparePaths(Map<String, Set<String>> expected, List<TPathChanges> actual) {
+  private boolean comparePaths(Map<String, Collection<String>> expected, List<TPathChanges> actual) {
     if (expected.size() != actual.size()) {
       return false;
     }
@@ -86,7 +86,7 @@ public class TestImageRetriever {
         return false;
       }
 
-      Set<String> expectedPaths = expected.get(pathChanges.getAuthzObj());
+      Collection<String> expectedPaths = expected.get(pathChanges.getAuthzObj());
       for (List<String> path : pathChanges.getAddPaths()) {
         if (!expectedPaths.contains(StringUtils.join(path, "/"))) {
           return false;

http://git-wip-us.apache.org/repos/asf/sentry/blob/7d18d1bf/sentry-hdfs/sentry-hdfs-service/src/test/java/org/apache/sentry/hdfs/TestSentryHDFSServiceProcessor.java
----------------------------------------------------------------------
diff --git a/sentry-hdfs/sentry-hdfs-service/src/test/java/org/apache/sentry/hdfs/TestSentryHDFSServiceProcessor.java b/sentry-hdfs/sentry-hdfs-service/src/test/java/org/apache/sentry/hdfs/TestSentryHDFSServiceProcessor.java
index e403d7c..4652dc9 100644
--- a/sentry-hdfs/sentry-hdfs-service/src/test/java/org/apache/sentry/hdfs/TestSentryHDFSServiceProcessor.java
+++ b/sentry-hdfs/sentry-hdfs-service/src/test/java/org/apache/sentry/hdfs/TestSentryHDFSServiceProcessor.java
@@ -52,7 +52,7 @@ public class TestSentryHDFSServiceProcessor {
     Mockito.when(sentryStoreMock.getLastProcessedImageID())
         .thenReturn(1L);
     Mockito.when(sentryStoreMock.retrieveFullPathsImage())
-        .thenReturn(new PathsImage(new HashMap<String, Set<String>>(), 1, 1));
+        .thenReturn(new PathsImage(new HashMap<String, Collection<String>>(), 1, 1));
 
     Mockito.when(sentryStoreMock.getLastProcessedPermChangeID())
         .thenReturn(1L);
@@ -77,7 +77,7 @@ public class TestSentryHDFSServiceProcessor {
     Mockito.when(sentryStoreMock.getLastProcessedImageID())
         .thenReturn(2L);
     Mockito.when(sentryStoreMock.retrieveFullPathsImage())
-        .thenReturn(new PathsImage(new HashMap<String, Set<String>>(), 3, 2));
+        .thenReturn(new PathsImage(new HashMap<String, Collection<String>>(), 3, 2));
 
     Mockito.when(sentryStoreMock.getLastProcessedPermChangeID())
         .thenReturn(3L);

http://git-wip-us.apache.org/repos/asf/sentry/blob/7d18d1bf/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MAuthzPathsMapping.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MAuthzPathsMapping.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MAuthzPathsMapping.java
index fee5279..c51f25a 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MAuthzPathsMapping.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MAuthzPathsMapping.java
@@ -19,6 +19,7 @@
 package org.apache.sentry.provider.db.service.model;
 
 import javax.jdo.annotations.PersistenceCapable;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.Set;
@@ -93,22 +94,21 @@ public class MAuthzPathsMapping {
     return null;
   }
 
-  /*
-  This method is used to get path set in string format
-  constructed from Set<MPath>.
-  */
-  public Set<String> getPathStrings() {
-    Set<String> paths = new HashSet<>(this.paths.size());
+  /**
+   * @return collection of paths strings contained in this object.
+   */
+  public Collection<String> getPathStrings() {
+    Collection<String> pathValues = new ArrayList<>(this.paths.size());
     for (MPath path : this.paths) {
-      paths.add(path.getPath());
+      pathValues.add(path.getPath());
     }
-    return paths;
+    return pathValues;
   }
 
   @Override
   public String toString() {
     return "MSentryPathsUpdate authzSnapshotID=[" + authzSnapshotID + "], authzObj=[" + authzObjName
-        + "], paths=[" + paths.toString() + "], createTimeMs=[" + String.valueOf(createTimeMs) + "]";
+        + "], paths=[" + paths.toString() + "], createTimeMs=[" + createTimeMs + "]";
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/sentry/blob/7d18d1bf/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/PathsImage.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/PathsImage.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/PathsImage.java
index 409a557..9813a5a 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/PathsImage.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/persistent/PathsImage.java
@@ -18,8 +18,8 @@
 
 package org.apache.sentry.provider.db.service.persistent;
 
+import java.util.Collection;
 import java.util.Map;
-import java.util.Set;
 
 /**
  * A container for complete hive paths snapshot.
@@ -30,11 +30,11 @@ import java.util.Set;
 public class PathsImage {
 
   // A full image of hiveObj to Paths mapping.
-  private final Map<String, Set<String>> pathImage;
+  private final Map<String, Collection<String>> pathImage;
   private final long id;
   private final long curImgNum;
 
-  public PathsImage(Map<String, Set<String>> pathImage, long id, long curImgNum) {
+  public PathsImage(Map<String, Collection<String>> pathImage, long id, long curImgNum) {
     this.pathImage = pathImage;
     this.id = id;
     this.curImgNum = curImgNum;
@@ -48,7 +48,7 @@ public class PathsImage {
     return curImgNum;
   }
 
-  public Map<String, Set<String>> getPathImage() {
+  public Map<String, Collection<String>> getPathImage() {
     return pathImage;
   }
 }

http://git-wip-us.apache.org/repos/asf/sentry/blob/7d18d1bf/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 04f6b43..8334034 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
@@ -2659,7 +2659,7 @@ public class SentryStore {
         pm.setDetachAllOnCommit(false); // No need to detach objects
         long curChangeID = getLastProcessedChangeIDCore(pm, MSentryPathChange.class);
         long curImageID = getCurrentAuthzPathsSnapshotID(pm);
-        Map<String, Set<String>> pathImage = retrieveFullPathsImageCore(pm, curImageID);
+        Map<String, Collection<String>> pathImage = retrieveFullPathsImageCore(pm, curImageID);
 
         return new PathsImage(pathImage, curChangeID, curImageID);
       }
@@ -2672,7 +2672,8 @@ public class SentryStore {
    *
    * @return a mapping of hiveObj to &lt Paths &gt.
    */
-  private Map<String, Set<String>> retrieveFullPathsImageCore(PersistenceManager pm, long currentSnapshotID) {
+  private Map<String, Collection<String>> retrieveFullPathsImageCore(PersistenceManager pm,
+                                                                     long currentSnapshotID) {
     if (currentSnapshotID <= EMPTY_PATHS_SNAPSHOT_ID) {
       return Collections.emptyMap();
     }
@@ -2687,7 +2688,7 @@ public class SentryStore {
       return Collections.emptyMap();
     }
 
-    Map<String, Set<String>> retVal = new HashMap<>(authzToPathsMappings.size());
+    Map<String, Collection<String>> retVal = new HashMap<>(authzToPathsMappings.size());
     for (MAuthzPathsMapping authzToPaths : authzToPathsMappings) {
       retVal.put(authzToPaths.getAuthzObjName(), authzToPaths.getPathStrings());
     }
@@ -2703,7 +2704,7 @@ public class SentryStore {
    * @param notificationID the latest notificationID associated with the snapshot
    * @throws Exception
    */
-  public void persistFullPathsImage(final Map<String, Set<String>> authzPaths,
+  public void persistFullPathsImage(final Map<String, Collection<String>> authzPaths,
       final long notificationID) throws Exception {
     tm.executeTransactionWithRetry(
       new TransactionBlock() {
@@ -2717,7 +2718,7 @@ public class SentryStore {
           long snapshotID = getCurrentAuthzPathsSnapshotID(pm);
           long nextSnapshotID = snapshotID + 1;
           pm.makePersistent(new MAuthzPathsSnapshotId(nextSnapshotID));
-          for (Map.Entry<String, Set<String>> authzPath : authzPaths.entrySet()) {
+          for (Map.Entry<String, Collection<String>> authzPath : authzPaths.entrySet()) {
             pm.makePersistent(new MAuthzPathsMapping(nextSnapshotID, authzPath.getKey(), authzPath.getValue()));
           }
           return null;

http://git-wip-us.apache.org/repos/asf/sentry/blob/7d18d1bf/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/FullUpdateInitializer.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/FullUpdateInitializer.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/FullUpdateInitializer.java
index 760a2b5..d62196f 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/FullUpdateInitializer.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/FullUpdateInitializer.java
@@ -449,7 +449,7 @@ public final class FullUpdateInitializer implements AutoCloseable {
    * @throws InterruptedException if processing was interrupted
    */
   @SuppressWarnings("squid:S00112")
-  Map<String, Set<String>> getFullHMSSnapshot() throws Exception {
+  Map<String, Collection<String>> getFullHMSSnapshot() throws Exception {
     // Get list of all HMS databases
     List<String> allDbStr;
     HMSClient c = null;
@@ -470,7 +470,7 @@ public final class FullUpdateInitializer implements AutoCloseable {
     }
 
     // Resulting full snapshot
-    Map<String, Set<String>> fullSnapshot = new HashMap<>();
+    Map<String, Collection<String>> fullSnapshot = new HashMap<>();
 
     // As async tasks complete, merge their results into full snapshot.
     while (!results.isEmpty()) {
@@ -491,7 +491,7 @@ public final class FullUpdateInitializer implements AutoCloseable {
       for (Map.Entry<String, Set<String>> entry: objectMapping.entrySet()) {
         String key = entry.getKey();
         Set<String> val = entry.getValue();
-        Set<String> existingSet = fullSnapshot.get(key);
+        Set<String> existingSet = (Set<String>)fullSnapshot.get(key);
         if (existingSet == null) {
           fullSnapshot.put(key, val);
           continue;

http://git-wip-us.apache.org/repos/asf/sentry/blob/7d18d1bf/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/FullUpdateModifier.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/FullUpdateModifier.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/FullUpdateModifier.java
index 2cd18ea..2501970 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/FullUpdateModifier.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/FullUpdateModifier.java
@@ -33,6 +33,7 @@ import org.apache.sentry.binding.metastore.messaging.json.SentryJSONDropTableMes
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -68,7 +69,7 @@ final class FullUpdateModifier {
    */
   // NOTE: we pass deserializer here instead of using built-in one to simplify testing.
   // Tests use mock serializers and thus we do not have to construct proper events.
-  static void applyEvent(Map<String, Set<String>> image, NotificationEvent event,
+  static void applyEvent(Map<String, Collection<String>> image, NotificationEvent event,
                          MessageDeserializer deserializer) {
     HCatEventMessage.EventType eventType =
             HCatEventMessage.EventType.valueOf(event.getEventType());
@@ -108,7 +109,7 @@ final class FullUpdateModifier {
   /**
    * Add mapping from the new database name to location {dbname: {location}}.
    */
-  private static void createDatabase(Map<String, Set<String>> image, NotificationEvent event,
+  private static void createDatabase(Map<String, Collection<String>> image, NotificationEvent event,
                                      MessageDeserializer deserializer) {
     SentryJSONCreateDatabaseMessage message =
             (SentryJSONCreateDatabaseMessage) deserializer
@@ -139,7 +140,7 @@ final class FullUpdateModifier {
     } else {
       // Sanity check the information and print warnings if database exists but
       // with a different location
-      Set<String> oldLocations = image.get(dbName);
+      Set<String> oldLocations = (Set<String>)image.get(dbName);
       LOGGER.debug("database {} already exists, ignored", dbName);
       if (!oldLocations.contains(location)) {
         LOGGER.warn("database {} exists but location is different from {}", dbName, location);
@@ -151,7 +152,7 @@ final class FullUpdateModifier {
    * Remove a mapping from database name and remove all mappings which look like dbName.tableName
    * where dbName matches database name.
    */
-  private static void dropDatabase(Map<String, Set<String>> image, NotificationEvent event,
+  private static void dropDatabase(Map<String, Collection<String>> image, NotificationEvent event,
                                    MessageDeserializer deserializer) {
     SentryJSONDropDatabaseMessage message =
             (SentryJSONDropDatabaseMessage) deserializer.getDropDatabaseMessage(event.getMessage());
@@ -174,7 +175,7 @@ final class FullUpdateModifier {
     }
 
     // If the database is alreday deleted, we have nothing to do
-    Set<String> locations = image.get(dbName);
+    Set<String> locations = (Set<String>)image.get(dbName);
     if (locations == null) {
       LOGGER.debug("database {} is already deleted", dbName);
       return;
@@ -193,9 +194,9 @@ final class FullUpdateModifier {
     String dbPrefix = dbName + ".";
 
     // Remove all objects for this database
-    for (Iterator<Map.Entry<String, Set<String>>> it = image.entrySet().iterator();
+    for (Iterator<Map.Entry<String, Collection<String>>> it = image.entrySet().iterator();
          it.hasNext(); ) {
-      Map.Entry<String, Set<String>> entry = it.next();
+      Map.Entry<String, Collection<String>> entry = it.next();
       String key = entry.getKey();
       if (key.startsWith(dbPrefix)) {
         LOGGER.debug("Removing {}", key);
@@ -207,7 +208,7 @@ final class FullUpdateModifier {
   /**
    * Add mapping for dbName.tableName.
    */
-  private static void createTable(Map<String, Set<String>> image, NotificationEvent event,
+  private static void createTable(Map<String, Collection<String>> image, NotificationEvent event,
                                   MessageDeserializer deserializer) {
     SentryJSONCreateTableMessage message = (SentryJSONCreateTableMessage) deserializer
             .getCreateTableMessage(event.getMessage());
@@ -244,7 +245,7 @@ final class FullUpdateModifier {
     } else {
       // Sanity check the information and print warnings if table exists but
       // with a different location
-      Set<String> oldLocations = image.get(authName);
+      Set<String> oldLocations = (Set<String>)image.get(authName);
       LOGGER.debug("Table {} already exists, ignored", authName);
       if (!oldLocations.contains(location)) {
         LOGGER.warn("Table {} exists but location is different from {}", authName, location);
@@ -255,7 +256,7 @@ final class FullUpdateModifier {
   /**
    * Drop mapping from dbName.tableName
    */
-  private static void dropTable(Map<String, Set<String>> image, NotificationEvent event,
+  private static void dropTable(Map<String, Collection<String>> image, NotificationEvent event,
                                 MessageDeserializer deserializer) {
     SentryJSONDropTableMessage message = (SentryJSONDropTableMessage) deserializer
             .getDropTableMessage(event.getMessage());
@@ -283,7 +284,7 @@ final class FullUpdateModifier {
     }
 
     String authName = dbName.toLowerCase() + "." + tableName.toLowerCase();
-    Set<String> locations = image.get(authName);
+    Set<String> locations = (Set<String>)image.get(authName);
     if (locations != null && locations.contains(path)) {
       LOGGER.debug("Removing {}", authName);
       image.remove(authName);
@@ -305,7 +306,7 @@ final class FullUpdateModifier {
    * </ul>
    *
    */
-  private static void alterTable(Map<String, Set<String>> image, NotificationEvent event,
+  private static void alterTable(Map<String, Collection<String>> image, NotificationEvent event,
                                  MessageDeserializer deserializer) {
     SentryJSONAlterTableMessage message =
             (SentryJSONAlterTableMessage) deserializer.getAlterTableMessage(event.getMessage());
@@ -362,7 +363,7 @@ final class FullUpdateModifier {
     if (!prevDbName.equals(newDbName)) {
       // Database name change
       LOGGER.debug("Changing database name: {} -> {}", prevDbName, newDbName);
-      Set<String> locations = image.get(prevDbName);
+      Set<String> locations = (Set<String>)image.get(prevDbName);
       if (locations != null) {
         // Rename database if it is not renamed yet
         if (!image.containsKey(newDbName)) {
@@ -383,7 +384,7 @@ final class FullUpdateModifier {
 
     if (!prevAuthName.equals(newAuthName)) {
       // Either the database name or table name changed, rename objects
-      Set<String> locations = image.get(prevAuthName);
+      Set<String> locations = (Set<String>)image.get(prevAuthName);
       if (locations != null) {
         // Rename if it is not renamed yet
         if (!image.containsKey(newAuthName)) {
@@ -401,7 +402,7 @@ final class FullUpdateModifier {
     if (!prevPath.equals(newPath)) {
       LOGGER.debug("Location change: {} -> {}", prevPath, newPath);
       // Location change
-      Set<String> locations = image.get(newAuthName);
+      Set<String> locations = (Set<String>) image.get(newAuthName);
       if (locations != null && locations.contains(prevPath) && !locations.contains(newPath)) {
         locations.remove(prevPath);
         locations.add(newPath);
@@ -415,7 +416,7 @@ final class FullUpdateModifier {
   /**
    * Add partition just adds a new location to the existing table.
    */
-  private static void addPartition(Map<String, Set<String>> image, NotificationEvent event,
+  private static void addPartition(Map<String, Collection<String>> image, NotificationEvent event,
                                    MessageDeserializer deserializer) {
     SentryJSONAddPartitionMessage message =
             (SentryJSONAddPartitionMessage) deserializer.getAddPartitionMessage(event.getMessage());
@@ -439,7 +440,7 @@ final class FullUpdateModifier {
       return;
     }
 
-    Set<String> oldLocations = image.get(authName);
+    Set<String> oldLocations = (Set<String>) image.get(authName);
     if (oldLocations == null) {
       LOGGER.warn("Add partition for {}: missing table locations",authName);
       return;
@@ -458,7 +459,7 @@ final class FullUpdateModifier {
   /**
    * Drop partition removes location from the existing table.
    */
-  private static void dropPartition(Map<String, Set<String>> image, NotificationEvent event,
+  private static void dropPartition(Map<String, Collection<String>> image, NotificationEvent event,
                                     MessageDeserializer deserializer) {
     SentryJSONDropPartitionMessage message =
             (SentryJSONDropPartitionMessage) deserializer
@@ -482,7 +483,7 @@ final class FullUpdateModifier {
       return;
     }
 
-    Set<String> oldLocations = image.get(authName);
+    Set<String> oldLocations = (Set<String>) image.get(authName);
     if (oldLocations == null) {
       LOGGER.warn("Add partition for {}: missing table locations",authName);
       return;
@@ -497,7 +498,7 @@ final class FullUpdateModifier {
     }
   }
 
-  private static void alterPartition(Map<String, Set<String>> image, NotificationEvent event,
+  private static void alterPartition(Map<String, Collection<String>> image, NotificationEvent event,
                                      MessageDeserializer deserializer) {
     SentryJSONAlterPartitionMessage message =
             (SentryJSONAlterPartitionMessage) deserializer
@@ -542,7 +543,7 @@ final class FullUpdateModifier {
       return;
     }
 
-    Set<String> locations = image.get(authName);
+    Set<String> locations = (Set<String>) image.get(authName);
     if (locations == null) {
       LOGGER.warn("Missing partition locations for {}", authName);
       return;
@@ -559,7 +560,7 @@ final class FullUpdateModifier {
    * Walk through the map and rename all instances of oldKey to newKey.
    */
   @VisibleForTesting
-  protected static void renamePrefixKeys(Map<String, Set<String>> image,
+  protected static void renamePrefixKeys(Map<String, Collection<String>> image,
                                          String oldKey, String newKey) {
     // The trick is that we can't just iterate through the map, remove old values and
     // insert new values. While we can remove old values with iterators,
@@ -567,15 +568,15 @@ final class FullUpdateModifier {
     // a new map and merge them in the end.
     Map<String, Set<String>> replacement = new HashMap<>();
 
-    for (Iterator<Map.Entry<String, Set<String>>> it = image.entrySet().iterator();
+    for (Iterator<Map.Entry<String, Collection<String>>> it = image.entrySet().iterator();
          it.hasNext(); ) {
-      Map.Entry<String, Set<String>> entry = it.next();
+      Map.Entry<String, Collection<String>> entry = it.next();
       String key = entry.getKey();
       if (key.startsWith(oldKey)) {
         String updatedKey = key.replaceAll("^" + oldKey + "(.*)", newKey + "$1");
         if (!image.containsKey(updatedKey)) {
           LOGGER.debug("Rename {} to {}", key, updatedKey);
-          replacement.put(updatedKey, entry.getValue());
+          replacement.put(updatedKey, (Set<String>) entry.getValue());
           it.remove();
         } else {
           LOGGER.warn("skipping key {} - already present", updatedKey);
@@ -593,7 +594,7 @@ final class FullUpdateModifier {
    * @param m1 source map
    * @param m2 map with replacement values
    */
-  private static void mergeMaps(Map<String, Set<String>> m1, Map<String, Set<String>> m2) {
+  private static void mergeMaps(Map<String, Collection<String>> m1, Map<String, Set<String>> m2) {
     // Merge replacement values into the original map but only if they are not
     // already there
     for (Map.Entry<String, Set<String>> entry : m2.entrySet()) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/7d18d1bf/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryHMSClient.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryHMSClient.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryHMSClient.java
index 12bf4a1..6ec163b 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryHMSClient.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/service/thrift/SentryHMSClient.java
@@ -18,21 +18,10 @@
 
 package org.apache.sentry.service.thrift;
 
-import static com.codahale.metrics.MetricRegistry.name;
-import static java.util.Collections.emptyMap;
-
 import com.codahale.metrics.Counter;
 import com.codahale.metrics.Timer;
 import com.codahale.metrics.Timer.Context;
-
 import com.google.common.annotations.VisibleForTesting;
-
-import java.io.IOException;
-
-import java.util.Collections;
-import java.util.Map;
-import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
@@ -43,12 +32,18 @@ import org.apache.sentry.binding.metastore.messaging.json.SentryJSONMessageDeser
 import org.apache.sentry.provider.db.service.persistent.PathsImage;
 import org.apache.sentry.provider.db.service.persistent.SentryStore;
 import org.apache.sentry.provider.db.service.thrift.SentryMetrics;
-
 import org.apache.thrift.TException;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+
+import static com.codahale.metrics.MetricRegistry.name;
+import static java.util.Collections.emptyMap;
+
 /**
  * Wrapper class for <Code>HiveMetaStoreClient</Code>
  *
@@ -145,13 +140,13 @@ class SentryHMSClient implements AutoCloseable {
   PathsImage getFullSnapshot() {
     if (client == null) {
       LOGGER.error(NOT_CONNECTED_MSG);
-      return new PathsImage(Collections.<String, Set<String>>emptyMap(),
+      return new PathsImage(Collections.<String, Collection<String>>emptyMap(),
           SentryStore.EMPTY_NOTIFICATION_ID, SentryStore.EMPTY_PATHS_SNAPSHOT_ID);
     }
 
     try {
       CurrentNotificationEventId eventIdBefore = client.getCurrentNotificationEventId();
-      Map<String, Set<String>> pathsFullSnapshot = fetchFullUpdate();
+      Map<String, Collection<String>> pathsFullSnapshot = fetchFullUpdate();
       if (pathsFullSnapshot.isEmpty()) {
         return new PathsImage(pathsFullSnapshot, SentryStore.EMPTY_NOTIFICATION_ID,
             SentryStore.EMPTY_PATHS_SNAPSHOT_ID);
@@ -184,7 +179,7 @@ class SentryHMSClient implements AutoCloseable {
         if (response == null || !response.isSetEvents() || response.getEvents().isEmpty()) {
           LOGGER.error("Snapshot discarded, updates to HMS data while shapshot is being taken."
                   + "ID Before: {}. ID After: {}", eventIdBefore.getEventId(), eventIdAfter.getEventId());
-          return new PathsImage(Collections.<String, Set<String>>emptyMap(),
+          return new PathsImage(Collections.<String, Collection<String>>emptyMap(),
                   SentryStore.EMPTY_NOTIFICATION_ID, SentryStore.EMPTY_PATHS_SNAPSHOT_ID);
         }
 
@@ -216,7 +211,7 @@ class SentryHMSClient implements AutoCloseable {
     } catch (TException failure) {
       LOGGER.error("Fetching a new HMS snapshot cannot continue because an error occurred during "
           + "the HMS communication: ", failure.getMessage());
-      return new PathsImage(Collections.<String, Set<String>>emptyMap(),
+      return new PathsImage(Collections.<String, Collection<String>>emptyMap(),
           SentryStore.EMPTY_NOTIFICATION_ID, SentryStore.EMPTY_PATHS_SNAPSHOT_ID);
     }
   }
@@ -227,12 +222,12 @@ class SentryHMSClient implements AutoCloseable {
    * @return HMS snapshot. Snapshot consists of a mapping from auth object name to the set of paths
    *     corresponding to that name.
    */
-  private Map<String, Set<String>> fetchFullUpdate() {
+  private Map<String, Collection<String>> fetchFullUpdate() {
     LOGGER.info("Request full HMS snapshot");
     try (FullUpdateInitializer updateInitializer =
              new FullUpdateInitializer(hiveConnectionFactory, conf);
          Context context = updateTimer.time()) {
-      Map<String, Set<String>> pathsUpdate = updateInitializer.getFullHMSSnapshot();
+      Map<String, Collection<String>> pathsUpdate = updateInitializer.getFullHMSSnapshot();
       LOGGER.info("Obtained full HMS snapshot");
       return pathsUpdate;
     } catch (Exception ignored) {

http://git-wip-us.apache.org/repos/asf/sentry/blob/7d18d1bf/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 3da6a4e..35417b7 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
@@ -19,22 +19,24 @@
 package org.apache.sentry.provider.db.service.persistent;
 
 import java.io.File;
-import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
-import java.util.List;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.collect.Lists;
+import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.alias.CredentialProvider;
@@ -2443,7 +2445,7 @@ public class TestSentryStore extends org.junit.Assert {
    */
   @Test
   public void testPersistFullPathsImage() throws Exception {
-    Map<String, Set<String>> authzPaths = new HashMap<>();
+    Map<String, Collection<String>> authzPaths = new HashMap<>();
     // Makes sure that authorizable object could be associated
     // with different paths and can be properly persisted into database.
     authzPaths.put("db1.table1", Sets.newHashSet("/user/hive/warehouse/db2.db/table1.1",
@@ -2459,21 +2461,21 @@ public class TestSentryStore extends org.junit.Assert {
     PathsImage pathsImage = sentryStore.retrieveFullPathsImage();
     long savedNotificationID = sentryStore.getLastProcessedNotificationID();
     assertEquals(1, pathsImage.getCurImgNum());
-    Map<String, Set<String>> pathImage = pathsImage.getPathImage();
+    Map<String, Collection<String>> pathImage = pathsImage.getPathImage();
     assertEquals(3, pathImage.size());
-    for (Map.Entry<String, Set<String>> entry : pathImage.entrySet()) {
+    for (Map.Entry<String, Collection<String>> entry : pathImage.entrySet()) {
       assertEquals(2, entry.getValue().size());
     }
     assertEquals(2, pathImage.get("db2.table2").size());
-    assertEquals(Sets.newHashSet("/user/hive/warehouse/db2.db/table1.1",
-                                "/user/hive/warehouse/db2.db/table1.2"),
-                                pathImage.get("db1.table1"));
-    assertEquals(Sets.newHashSet("/user/hive/warehouse/db2.db/table2.1",
-                                "/user/hive/warehouse/db2.db/table2.2"),
-                                pathImage.get("db1.table2"));
-    assertEquals(Sets.newHashSet("/user/hive/warehouse/db2.db/table2.1",
-                                "/user/hive/warehouse/db2.db/table2.3"),
-                                pathImage.get("db2.table2"));
+    assertTrue(CollectionUtils.isEqualCollection(Lists.newArrayList("/user/hive/warehouse/db2.db/table1.1",
+            "/user/hive/warehouse/db2.db/table1.2"),
+            pathImage.get("db1.table1")));
+    assertTrue(CollectionUtils.isEqualCollection(Lists.newArrayList("/user/hive/warehouse/db2.db/table2.1",
+            "/user/hive/warehouse/db2.db/table2.2"),
+            pathImage.get("db1.table2")));
+    assertTrue(CollectionUtils.isEqualCollection(Lists.newArrayList("/user/hive/warehouse/db2.db/table2.1",
+            "/user/hive/warehouse/db2.db/table2.3"),
+            pathImage.get("db2.table2")));
     assertEquals(6, sentryStore.getMPaths().size());
     assertEquals(notificationID, savedNotificationID);
   }
@@ -2483,7 +2485,7 @@ public class TestSentryStore extends org.junit.Assert {
     long notificationID = 1;
 
     // Persist an empty image so that we can add paths to it.
-    sentryStore.persistFullPathsImage(new HashMap<String, Set<String>>(), 0);
+    sentryStore.persistFullPathsImage(new HashMap<String, Collection<String>>(), 0);
 
     // Create two path updates with the same sequence ID
     UniquePathsUpdate update1 = new UniquePathsUpdate("u1", notificationID, false);
@@ -2503,7 +2505,7 @@ public class TestSentryStore extends org.junit.Assert {
     assertEquals(notificationID, latestID);
 
     // Check that retrieving a full paths image returns both paths updates
-    Map<String, Set<String>> pathsImage = sentryStore.retrieveFullPathsImage().getPathImage();
+    Map<String, Collection<String>> pathsImage = sentryStore.retrieveFullPathsImage().getPathImage();
     assertEquals(2, pathsImage.size());
     assertEquals(1, pathsImage.get("db1").size());
     assertTrue(pathsImage.get("db1").contains("/hive/db1"));
@@ -2541,7 +2543,7 @@ public class TestSentryStore extends org.junit.Assert {
     long notificationID = 0;
 
     // Persist an empty image so that we can add paths to it.
-    sentryStore.persistFullPathsImage(new HashMap<String, Set<String>>(), notificationID);
+    sentryStore.persistFullPathsImage(new HashMap<String, Collection<String>>(), notificationID);
 
     // Add "db1.table1" authzObj
     Long lastNotificationId = sentryStore.getLastProcessedNotificationID();
@@ -2554,7 +2556,7 @@ public class TestSentryStore extends org.junit.Assert {
     sentryStore.addAuthzPathsMapping("db1.table",
           Sets.newHashSet("db1/tbl1", "db1/tbl2"), addUpdate);
     PathsImage pathsImage = sentryStore.retrieveFullPathsImage();
-    Map<String, Set<String>> pathImage = pathsImage.getPathImage();
+    Map<String, Collection<String>> pathImage = pathsImage.getPathImage();
     assertEquals(1, pathImage.size());
     assertEquals(2, pathImage.get("db1.table").size());
     assertEquals(2, sentryStore.getMPaths().size());
@@ -2604,13 +2606,13 @@ public class TestSentryStore extends org.junit.Assert {
 
   @Test
   public void testRenameUpdateAuthzPathsMapping() throws Exception {
-    Map<String, Set<String>> authzPaths = new HashMap<>();
+    Map<String, Collection<String>> authzPaths = new HashMap<>();
     Long lastNotificationId = sentryStore.getLastProcessedNotificationID();
     authzPaths.put("db1.table1", Sets.newHashSet("user/hive/warehouse/db1.db/table1",
                                                 "user/hive/warehouse/db1.db/table1/p1"));
     authzPaths.put("db1.table2", Sets.newHashSet("user/hive/warehouse/db1.db/table2"));
     sentryStore.persistFullPathsImage(authzPaths, lastNotificationId);
-    Map<String, Set<String>> pathsImage = sentryStore.retrieveFullPathsImage().getPathImage();
+    Map<String, Collection<String>> pathsImage = sentryStore.retrieveFullPathsImage().getPathImage();
     assertEquals(2, pathsImage.size());
 
 
@@ -2626,9 +2628,9 @@ public class TestSentryStore extends org.junit.Assert {
     assertEquals(2, pathsImage.size());
     assertEquals(3, sentryStore.getMPaths().size());
     assertTrue(pathsImage.containsKey("db1.newTable1"));
-    assertEquals(Sets.newHashSet("user/hive/warehouse/db1.db/table1/p1",
-                                "user/hive/warehouse/db1.db/newTable1"),
-                  pathsImage.get("db1.newTable1"));
+    assertTrue(CollectionUtils.isEqualCollection(Lists.newArrayList("user/hive/warehouse/db1.db/table1/p1",
+            "user/hive/warehouse/db1.db/newTable1"),
+            pathsImage.get("db1.newTable1")));
 
     // Query the persisted path change and ensure it equals to the original one
     long lastChangeID = sentryStore.getLastProcessedPathChangeID();
@@ -2647,9 +2649,9 @@ public class TestSentryStore extends org.junit.Assert {
     assertEquals(2, pathsImage.size());
     assertEquals(3, sentryStore.getMPaths().size());
     assertTrue(pathsImage.containsKey("db1.newTable2"));
-    assertEquals(Sets.newHashSet("user/hive/warehouse/db1.db/table1/p1",
-                                "user/hive/warehouse/db1.db/newTable1"),
-                  pathsImage.get("db1.newTable2"));
+    assertTrue(CollectionUtils.isEqualCollection(Lists.newArrayList("user/hive/warehouse/db1.db/table1/p1",
+            "user/hive/warehouse/db1.db/newTable1"),
+            pathsImage.get("db1.newTable2")));
     lastNotificationId = sentryStore.getLastProcessedNotificationID();
     assertEquals(2, lastNotificationId.longValue());
 
@@ -2672,9 +2674,9 @@ public class TestSentryStore extends org.junit.Assert {
     assertEquals(2, pathsImage.size());
     assertEquals(3, sentryStore.getMPaths().size());
     assertTrue(pathsImage.containsKey("db1.newTable2"));
-    assertEquals(Sets.newHashSet("user/hive/warehouse/db1.db/table1/p1",
-                                "user/hive/warehouse/db1.db/newTable2"),
-                  pathsImage.get("db1.newTable2"));
+    assertTrue(CollectionUtils.isEqualCollection(Lists.newArrayList("user/hive/warehouse/db1.db/table1/p1",
+            "user/hive/warehouse/db1.db/newTable2"),
+            pathsImage.get("db1.newTable2")));
 
     // Query the persisted path change and ensure it equals to the original one
     lastChangeID = sentryStore.getLastProcessedPathChangeID();
@@ -2686,7 +2688,7 @@ public class TestSentryStore extends org.junit.Assert {
 
   @Test
   public void testPersistAndReplaceANewPathsImage() throws Exception {
-    Map<String, Set<String>> authzPaths = new HashMap<>();
+    Map<String, Collection<String>> authzPaths = new HashMap<>();
     PathsImage pathsImage;
     long notificationID = 1;
 
@@ -2711,24 +2713,24 @@ public class TestSentryStore extends org.junit.Assert {
 
     pathsImage = sentryStore.retrieveFullPathsImage();
     assertEquals(2, pathsImage.getCurImgNum());
-    Map<String, Set<String>> pathImage = pathsImage.getPathImage();
+    Map<String, Collection<String>> pathImage = pathsImage.getPathImage();
     assertEquals(3, pathImage.size());
 
-    for (Map.Entry<String, Set<String>> entry : pathImage.entrySet()) {
+    for (Map.Entry<String, Collection<String>> entry : pathImage.entrySet()) {
       assertEquals(2, entry.getValue().size());
     }
 
     assertEquals(2, pathImage.get("db4.table2").size());
+    assertTrue(CollectionUtils.isEqualCollection(Lists.newArrayList("/another-warehouse/db2.db/table1.1",
+            "/another-warehouse/db2.db/table1.2"),
+            pathImage.get("db3.table1")));
+    assertTrue(CollectionUtils.isEqualCollection(Lists.newArrayList("/another-warehouse/db2.db/table2.1",
+            "/another-warehouse/db2.db/table2.2"),
+            pathImage.get("db3.table2")));
 
-    assertEquals(Sets.newHashSet("/another-warehouse/db2.db/table1.1",
-        "/another-warehouse/db2.db/table1.2"),
-        pathImage.get("db3.table1"));
-    assertEquals(Sets.newHashSet("/another-warehouse/db2.db/table2.1",
-        "/another-warehouse/db2.db/table2.2"),
-        pathImage.get("db3.table2"));
-    assertEquals(Sets.newHashSet("/another-warehouse/db2.db/table2.1",
-        "/another-warehouse/db2.db/table2.3"),
-        pathImage.get("db4.table2"));
+    assertTrue(CollectionUtils.isEqualCollection(Lists.newArrayList("/another-warehouse/db2.db/table2.1",
+            "/another-warehouse/db2.db/table2.3"),
+            pathImage.get("db4.table2")));
 
     assertEquals(6, sentryStore.getMPaths().size());
   }
@@ -2745,7 +2747,7 @@ public class TestSentryStore extends org.junit.Assert {
 
     // Persist a new image that contains a new image ID (it replaces previous paths)
     notificationID ++;
-    Map<String, Set<String>> authzPaths = new HashMap<>();
+    Map<String, Collection<String>> authzPaths = new HashMap<>();
     authzPaths.put("db2.table3", Sets.newHashSet("/user/hive/warehouse/db2.db/table1.1",
         "/user/hive/warehouse/db2.db/table1.2"));
     sentryStore.persistFullPathsImage(authzPaths, notificationID);
@@ -2757,7 +2759,7 @@ public class TestSentryStore extends org.junit.Assert {
     newAddUpdate.newPathChange("db2.table").addToAddPaths(Arrays.asList("db2", "tbl2"));
     sentryStore.addAuthzPathsMapping("db2.table", Sets.newHashSet("db2/tbl1", "db2/tbl2"), newAddUpdate);
     PathsImage pathsImage = sentryStore.retrieveFullPathsImage();
-    Map<String, Set<String>> pathImage = pathsImage.getPathImage();
+    Map<String, Collection<String>> pathImage = pathsImage.getPathImage();
     assertEquals(2, pathImage.size());
     assertEquals(2, pathImage.get("db2.table").size());
     assertEquals(4, sentryStore.getMPaths().size());
@@ -2781,7 +2783,7 @@ public class TestSentryStore extends org.junit.Assert {
   public void testRenameUpdateAfterReplacingANewPathsImage() throws Exception {
     long notificationID = 1;
 
-    Map<String, Set<String>> authzPaths = new HashMap<>();
+    Map<String, Collection<String>> authzPaths = new HashMap<>();
     // First image to persist (this will be replaced later)
     authzPaths.put("db1.table1", Sets.newHashSet("/user/hive/warehouse/db2.db/table1.1",
         "/user/hive/warehouse/db2.db/table1.2"));
@@ -2807,13 +2809,13 @@ public class TestSentryStore extends org.junit.Assert {
         .addToAddPaths(Arrays.asList("user", "hive", "warehouse", "db1.db", "newTable1"));
     sentryStore.renameAuthzPathsMapping("db3.table1", "db1.newTable1",
         "/another-warehouse/db3.db/table1.1", "user/hive/warehouse/db1.db/newTable1", renameUpdate);
-    Map<String, Set<String>> pathsImage = sentryStore.retrieveFullPathsImage().getPathImage();
+    Map<String, Collection<String>> pathsImage = sentryStore.retrieveFullPathsImage().getPathImage();
     assertEquals(2, pathsImage.size());
     assertEquals(4, sentryStore.getMPaths().size());
     assertTrue(pathsImage.containsKey("db1.newTable1"));
-    assertEquals(Sets.newHashSet("/another-warehouse/db3.db/table1.2",
-        "user/hive/warehouse/db1.db/newTable1"),
-        pathsImage.get("db1.newTable1"));
+    assertTrue(CollectionUtils.isEqualCollection(Lists.newArrayList("/another-warehouse/db3.db/table1.2",
+            "user/hive/warehouse/db1.db/newTable1"),
+            pathsImage.get("db1.newTable1")));
 
     // Update path of 'db1.newTable2' from 'db1.newTable1' to 'db1.newTable2'
     notificationID++;
@@ -2830,7 +2832,7 @@ public class TestSentryStore extends org.junit.Assert {
     assertEquals(3, pathsImage.size());
     assertEquals(5, sentryStore.getMPaths().size());
     assertTrue(pathsImage.containsKey("db1.newTable2"));
-    assertEquals(Sets.newHashSet("user/hive/warehouse/db1.db/newTable2"),
+    assertEquals(Lists.newArrayList("user/hive/warehouse/db1.db/newTable2"),
         pathsImage.get("db1.newTable2"));
   }
 
@@ -3259,7 +3261,7 @@ public class TestSentryStore extends org.junit.Assert {
 
   @Test
   public void testDuplicateNotification() throws Exception {
-    Map<String, Set<String>> authzPaths = new HashMap<>();
+    Map<String, Collection<String>> authzPaths = new HashMap<>();
     Long lastNotificationId = sentryStore.getLastProcessedNotificationID();
 
     lastNotificationId ++;
@@ -3267,7 +3269,7 @@ public class TestSentryStore extends org.junit.Assert {
       "user/hive/warehouse/db1.db/table1/p1"));
     authzPaths.put("db1.table2", Sets.newHashSet("user/hive/warehouse/db1.db/table2"));
     sentryStore.persistFullPathsImage(authzPaths, lastNotificationId);
-    Map<String, Set<String>> pathsImage = sentryStore.retrieveFullPathsImage().getPathImage();
+    Map<String, Collection<String>> pathsImage = sentryStore.retrieveFullPathsImage().getPathImage();
     assertEquals(2, pathsImage.size());
 
     if (lastNotificationId == null) {
@@ -3287,9 +3289,9 @@ public class TestSentryStore extends org.junit.Assert {
     assertEquals(2, pathsImage.size());
     assertEquals(3, sentryStore.getMPaths().size());
     assertTrue(pathsImage.containsKey("db1.newTable1"));
-    assertEquals(Sets.newHashSet("user/hive/warehouse/db1.db/table1/p1",
-      "user/hive/warehouse/db1.db/newTable1"),
-      pathsImage.get("db1.newTable1"));
+    assertTrue(CollectionUtils.isEqualCollection(Lists.newArrayList("user/hive/warehouse/db1.db/table1/p1",
+            "user/hive/warehouse/db1.db/newTable1"),
+            pathsImage.get("db1.newTable1")));
 
     // Query the persisted path change and ensure it equals to the original one
     long lastChangeID = sentryStore.getLastProcessedPathChangeID();
@@ -3320,13 +3322,13 @@ public class TestSentryStore extends org.junit.Assert {
       addToAddPaths(Arrays.asList("db1", "tbl2"));
 
     // Persist an empty image so that we can add paths to it.
-    sentryStore.persistFullPathsImage(new HashMap<String, Set<String>>(), 0);
+    sentryStore.persistFullPathsImage(new HashMap<String, Collection<String>>(), 0);
 
     assertEquals(sentryStore.isAuthzPathsMappingEmpty(), true);
     sentryStore.addAuthzPathsMapping("db1.table",
       Sets.newHashSet("db1/tbl1", "db1/tbl2"), addUpdate);
     PathsImage pathsImage = sentryStore.retrieveFullPathsImage();
-    Map<String, Set<String>> pathImage = pathsImage.getPathImage();
+    Map<String, Collection<String>> pathImage = pathsImage.getPathImage();
     assertEquals(1, pathImage.size());
     assertEquals(2, pathImage.get("db1.table").size());
     assertEquals(2, sentryStore.getMPaths().size());
@@ -3344,7 +3346,7 @@ public class TestSentryStore extends org.junit.Assert {
     SentryStore localSentryStore = new SentryStore(conf);
 
     // Persist an empty image so that we can add paths to it.
-    localSentryStore.persistFullPathsImage(new HashMap<String, Set<String>>(), 0);
+    localSentryStore.persistFullPathsImage(new HashMap<String, Collection<String>>(), 0);
 
     // Add "db1.table1" authzObj
     Long lastNotificationId = sentryStore.getLastProcessedNotificationID();
@@ -3357,7 +3359,7 @@ public class TestSentryStore extends org.junit.Assert {
     localSentryStore.addAuthzPathsMapping("db1.table",
         Sets.newHashSet("db1/tbl1", "db1/tbl2"), addUpdate);
     PathsImage pathsImage = localSentryStore.retrieveFullPathsImage();
-    Map<String, Set<String>> pathImage = pathsImage.getPathImage();
+    Map<String, Collection<String>> pathImage = pathsImage.getPathImage();
     assertEquals(1, pathImage.size());
     assertEquals(2, pathImage.get("db1.table").size());
     assertEquals(2, localSentryStore.getMPaths().size());

http://git-wip-us.apache.org/repos/asf/sentry/blob/7d18d1bf/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestFullUpdateInitializer.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestFullUpdateInitializer.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestFullUpdateInitializer.java
index 9d7bddd..589acbe 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestFullUpdateInitializer.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestFullUpdateInitializer.java
@@ -244,7 +244,7 @@ public class TestFullUpdateInitializer {
     HiveSnapshot snap = new HiveSnapshot().add(db1).add(db2).add(db3);
     MockClient c = new MockClient(snap);
 
-    Map<String, Set<String>> update;
+    Map<String, Collection<String>> update;
     try(FullUpdateInitializer cacheInitializer =
                 new FullUpdateInitializer(new MockHMSClientFactory(c), conf)) {
       update = cacheInitializer.getFullHMSSnapshot();
@@ -288,7 +288,7 @@ public class TestFullUpdateInitializer {
             thenReturn(Lists.newArrayList("tab1", "tab12"));
 
 
-    Map<String, Set<String>> update;
+    Map<String, Collection<String>> update;
     try(FullUpdateInitializer cacheInitializer =
                 new FullUpdateInitializer(new MockHMSClientFactory(client), conf)) {
       update = cacheInitializer.getFullHMSSnapshot();
@@ -319,7 +319,7 @@ public class TestFullUpdateInitializer {
       snap.add(db);
     }
     MockClient c = new MockClient(snap);
-    Map<String, Set<String>> update;
+    Map<String, Collection<String>> update;
     try(FullUpdateInitializer cacheInitializer =
                 new FullUpdateInitializer(new MockHMSClientFactory(c), conf)) {
       update = cacheInitializer.getFullHMSSnapshot();

http://git-wip-us.apache.org/repos/asf/sentry/blob/7d18d1bf/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestFullUpdateModifier.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestFullUpdateModifier.java b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestFullUpdateModifier.java
index 7deccb0..a8fcbf8 100644
--- a/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestFullUpdateModifier.java
+++ b/sentry-provider/sentry-provider-db/src/test/java/org/apache/sentry/service/thrift/TestFullUpdateModifier.java
@@ -32,6 +32,7 @@ import org.apache.sentry.binding.metastore.messaging.json.SentryJSONMessageDeser
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -64,7 +65,7 @@ public class TestFullUpdateModifier {
    */
   @Test
   public void testCreateDatabase() throws Exception {
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     NotificationEvent event = new NotificationEvent(0, 0, CREATE_DATABASE.toString(), "");
     MessageDeserializer deserializer = Mockito.mock(SentryJSONMessageDeserializer.class);
 
@@ -83,7 +84,7 @@ public class TestFullUpdateModifier {
    */
   @Test
   public void testDropDatabase() throws Exception {
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     update.put(DB.toLowerCase(), Collections.singleton(PATH));
     NotificationEvent event = new NotificationEvent(0, 0, DROP_DATABASE.toString(), "");
     MessageDeserializer deserializer = Mockito.mock(SentryJSONMessageDeserializer.class);
@@ -102,7 +103,7 @@ public class TestFullUpdateModifier {
    */
   @Test
   public void testDropDatabaseWrongLocation() throws Exception {
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     update.put(DB.toLowerCase(), Collections.singleton(PATH));
 
     NotificationEvent event = new NotificationEvent(0, 0, DROP_DATABASE.toString(), "");
@@ -126,7 +127,7 @@ public class TestFullUpdateModifier {
    */
   @Test
   public void testDropDatabaseWithTables() throws Exception {
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     update.put(DB.toLowerCase(), Collections.singleton(PATH));
     update.put(AUTH, Collections.singleton(PATH));
     update.put("unrelated", Collections.singleton(PATH));
@@ -149,7 +150,7 @@ public class TestFullUpdateModifier {
    */
   @Test
   public void testCreateTable() throws Exception {
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     NotificationEvent event = new NotificationEvent(0, 0, CREATE_TABLE.toString(), "");
     MessageDeserializer deserializer = Mockito.mock(SentryJSONMessageDeserializer.class);
 
@@ -168,7 +169,7 @@ public class TestFullUpdateModifier {
    */
   @Test
   public void testDropTable() throws Exception {
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     update.put(AUTH, Collections.singleton(PATH));
     NotificationEvent event = new NotificationEvent(0, 0, DROP_TABLE.toString(), "");
     MessageDeserializer deserializer = Mockito.mock(SentryJSONMessageDeserializer.class);
@@ -186,7 +187,7 @@ public class TestFullUpdateModifier {
    */
   @Test
   public void testDropTableWrongLocation() throws Exception {
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     update.put(AUTH, Collections.singleton(PATH));
     NotificationEvent event = new NotificationEvent(0, 0, DROP_TABLE.toString(), "");
     MessageDeserializer deserializer = Mockito.mock(SentryJSONMessageDeserializer.class);
@@ -208,7 +209,7 @@ public class TestFullUpdateModifier {
    */
   @Test
   public void testAddPartition() throws Exception {
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     Set<String> locations = new HashSet<>();
     locations.add(PATH);
     update.put(AUTH, locations);
@@ -239,7 +240,7 @@ public class TestFullUpdateModifier {
   public void testDropPartitions() throws Exception {
     String partPath = "hello/world";
     String partLocation = uri(partPath);
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     Set<String> locations = new HashSet<>();
     locations.add(PATH);
     locations.add(partPath);
@@ -268,7 +269,7 @@ public class TestFullUpdateModifier {
     String newPath = "better/world";
     String newLocation = uri(newPath);
 
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     Set<String> locations = new HashSet<>();
     locations.add(PATH);
     locations.add(partPath);
@@ -297,7 +298,7 @@ public class TestFullUpdateModifier {
    */
   @Test
   public void testAlterTableChangeDbNameNoTables() throws Exception {
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     update.put(DB.toLowerCase(), Collections.singleton(PATH));
     String newDbName = "Db2";
 
@@ -324,7 +325,7 @@ public class TestFullUpdateModifier {
    * @throws Exception
    */
   public void testAlterTableChangeDbNameWithTables() throws Exception {
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     update.put(DB.toLowerCase(), Collections.singleton(PATH));
     Set<String> locations = new HashSet<>(1);
     locations.add(PATH);
@@ -357,7 +358,7 @@ public class TestFullUpdateModifier {
    */
   @Test
   public void testAlterTableChangeTableName() throws Exception {
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     update.put(DB.toLowerCase(), Collections.singleton(PATH));
     Set<String> locations = new HashSet<>(1);
     locations.add(PATH);
@@ -390,7 +391,7 @@ public class TestFullUpdateModifier {
    */
   @Test
   public void testAlterTableChangeLocation() throws Exception {
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     update.put(DB.toLowerCase(), Collections.singleton(PATH));
     Set<String> locations = new HashSet<>(1);
     locations.add(PATH);
@@ -427,7 +428,7 @@ public class TestFullUpdateModifier {
     String oldKey = "foo.";
     String newKey = "baz.";
     String postfix = "bar";
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     update.put(oldKey + postfix , Collections.<String>emptySet());
     FullUpdateModifier.renamePrefixKeys(update, oldKey, newKey);
     assertEquals(1, update.size());
@@ -442,10 +443,10 @@ public class TestFullUpdateModifier {
    */
   @Test
   public void testRenameKeysWithConflicts() throws Exception {
-    Map<String, Set<String>> update = new HashMap<>();
+    Map<String, Collection<String>> update = new HashMap<>();
     update.put("foo.bar", Collections.<String>emptySet());
     update.put("baz.bar", Collections.<String>emptySet());
-    Map<String, Set<String>> expected = new HashMap<>(update);
+    Map<String, Collection<String>> expected = new HashMap<>(update);
 
     FullUpdateModifier.renamePrefixKeys(update, "foo.", "baz.");
     assertEquals(update, expected);

http://git-wip-us.apache.org/repos/asf/sentry/blob/7d18d1bf/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 35f8316..93afb61 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
@@ -25,11 +25,12 @@ import com.google.common.collect.Sets;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
+
 import junit.framework.Assert;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -103,7 +104,7 @@ public class TestHMSFollower {
     final long HMS_PROCESSED_EVENT_ID = 1L;
 
     // Mock that returns a full snapshot
-    Map<String, Set<String>> snapshotObjects = new HashMap<>();
+    Map<String, Collection<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);
@@ -155,7 +156,7 @@ public class TestHMSFollower {
     final long HMS_PROCESSED_EVENT_ID = 1L;
 
     // Mock that returns a full snapshot
-    Map<String, Set<String>> snapshotObjects = new HashMap<>();
+    Map<String, Collection<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);
@@ -207,7 +208,7 @@ public class TestHMSFollower {
     final long HMS_PROCESSED_EVENT_ID = 5L;
 
     // Mock that returns a full snapshot
-    Map<String, Set<String>> snapshotObjects = new HashMap<>();
+    Map<String, Collection<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);
@@ -797,7 +798,7 @@ public class TestHMSFollower {
     final long HMS_PROCESSED_EVENT_ID = 1L;
 
     // Mock that returns a full snapshot
-    Map<String, Set<String>> snapshotObjects = new HashMap<>();
+    Map<String, Collection<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);