You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by sr...@apache.org on 2014/05/31 04:25:04 UTC

[2/2] git commit: SENTRY-157: Support filter pushdown in DB Store client to reduce data transfer from DB Store service ( Arun Suresh via Sravya Tirukkovalur)

SENTRY-157: Support filter pushdown in DB Store client to reduce data transfer from DB Store service ( Arun Suresh via Sravya Tirukkovalur)


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

Branch: refs/heads/master
Commit: a7b45622309cc1753fa082be0c52d5e3e90faa42
Parents: a989b17
Author: Sravya Tirukkovalur <sr...@clouera.com>
Authored: Fri May 30 19:24:15 2014 -0700
Committer: Sravya Tirukkovalur <sr...@clouera.com>
Committed: Fri May 30 19:24:15 2014 -0700

----------------------------------------------------------------------
 .../hive/ql/exec/SentryGrantRevokeTask.java     |   2 +-
 .../sentry/policy/common/PolicyEngine.java      |  18 +-
 .../sentry/policy/db/SimpleDBPolicyEngine.java  |  14 +-
 .../db/AbstractTestSimplePolicyEngine.java      |  14 +-
 .../policy/db/TestPolicyParsingNegative.java    |  20 +-
 .../policy/db/TestSimpleDBPolicyEngineDFS.java  |   2 +-
 .../policy/search/SimpleSearchPolicyEngine.java |  13 +-
 .../cache/SimpleCacheProviderBackend.java       |   3 +-
 .../sentry/provider/common/ProviderBackend.java |   3 +-
 .../common/ResourceAuthorizationProvider.java   |  10 +-
 .../provider/common/TestGetGroupMapping.java    |  14 +-
 ...TListSentryPrivilegesForProviderRequest.java | 117 ++-
 .../thrift/TListSentryPrivilegesRequest.java    | 123 +++-
 .../db/service/thrift/TSentryAuthorizable.java  | 707 +++++++++++++++++++
 .../provider/db/SimpleDBProviderBackend.java    |   8 +-
 .../db/service/model/MSentryPrivilege.java      |   2 +-
 .../db/service/persistent/SentryStore.java      | 178 ++++-
 .../thrift/SentryPolicyServiceClient.java       |  53 +-
 .../thrift/SentryPolicyStoreProcessor.java      |   9 +-
 .../main/resources/sentry_policy_service.thrift |  11 +-
 .../db/service/persistent/TestSentryStore.java  |  28 +-
 .../thrift/TestSentryServiceIntegration.java    | 136 +++-
 .../file/SimpleFileProviderBackend.java         |   5 +-
 .../tests/e2e/dbprovider/TestDbEndToEnd.java    |  31 +-
 24 files changed, 1384 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
index f1e6247..1012605 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
@@ -257,7 +257,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
 
     try {
       if (hiveObjectDesc == null) {
-        privileges = sentryClient.listPrivilegesByRoleName(subject, principalName);
+        privileges = sentryClient.listAllPrivilegesByRoleName(subject, principalName);
         writeToFile(writeGrantInfo(privileges, principalName), desc.getResFile());
         return RETURN_CODE_SUCCESS;
       } else {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PolicyEngine.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PolicyEngine.java b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PolicyEngine.java
index c378a38..38a5b65 100644
--- a/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PolicyEngine.java
+++ b/sentry-policy/sentry-policy-common/src/main/java/org/apache/sentry/policy/common/PolicyEngine.java
@@ -17,11 +17,13 @@
 
 package org.apache.sentry.policy.common;
 
+import java.util.List;
 import java.util.Set;
 
 import javax.annotation.concurrent.ThreadSafe;
 
 import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.SentryConfigurationException;
 
 import com.google.common.collect.ImmutableSet;
@@ -45,9 +47,23 @@ public interface PolicyEngine {
    * type of privileges used in a policy file. Additionally it is possible further
    * processing of the privileges is needed before resolving to a privilege object.
    * @param group name
+   * @param active role-set
    * @return non-null immutable set of privileges
    */
-  public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet)
+  public ImmutableSet<String> getAllPrivileges(Set<String> groups, ActiveRoleSet roleSet)
+      throws SentryConfigurationException;
+
+  /**
+   * Get privileges associated with a group. Returns Strings which can be resolved
+   * by the caller. Strings are returned to separate the PolicyFile class from the
+   * type of privileges used in a policy file. Additionally it is possible further
+   * processing of the privileges is needed before resolving to a privilege object.
+   * @param group name
+   * @param active role-set
+   * @param authorizable Hierarchy (Can be null)
+   * @return non-null immutable set of privileges
+   */
+  public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet, Authorizable... authorizableHierarchy)
       throws SentryConfigurationException;
 
   public void close();

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/SimpleDBPolicyEngine.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/SimpleDBPolicyEngine.java b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/SimpleDBPolicyEngine.java
index a95ef7b..a03794e 100644
--- a/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/SimpleDBPolicyEngine.java
+++ b/sentry-policy/sentry-policy-db/src/main/java/org/apache/sentry/policy/db/SimpleDBPolicyEngine.java
@@ -16,9 +16,11 @@
  */
 package org.apache.sentry.policy.db;
 
+import java.util.List;
 import java.util.Set;
 
 import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.SentryConfigurationException;
 import org.apache.sentry.policy.common.PrivilegeFactory;
 import org.apache.sentry.policy.common.PolicyEngine;
@@ -56,16 +58,24 @@ public class SimpleDBPolicyEngine implements PolicyEngine {
     return new DBWildcardPrivilege.DBWildcardPrivilegeFactory();
   }
 
+
+
+  @Override
+  public ImmutableSet<String> getAllPrivileges(Set<String> groups,
+      ActiveRoleSet roleSet) throws SentryConfigurationException {
+    return getPrivileges(groups, roleSet, null);
+  }
+
   /**
    * {@inheritDoc}
    */
   @Override
-  public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet)
+  public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet, Authorizable... authorizableHierarchy)
       throws SentryConfigurationException {
     if(LOGGER.isDebugEnabled()) {
       LOGGER.debug("Getting permissions for {}", groups);
     }
-    ImmutableSet<String> result = providerBackend.getPrivileges(groups, roleSet);
+    ImmutableSet<String> result = providerBackend.getPrivileges(groups, roleSet, authorizableHierarchy);
     if(LOGGER.isDebugEnabled()) {
       LOGGER.debug("result = " + result);
     }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/AbstractTestSimplePolicyEngine.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/AbstractTestSimplePolicyEngine.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/AbstractTestSimplePolicyEngine.java
index 4625d6f..d1151e3 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/AbstractTestSimplePolicyEngine.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/AbstractTestSimplePolicyEngine.java
@@ -89,7 +89,7 @@ public abstract class AbstractTestSimplePolicyEngine {
         PERM_SERVER1_CUSTOMERS_DB_CUSTOMERS_PARTIAL_SELECT
         ));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPrivileges(set("manager"), ActiveRoleSet.ALL))
+        new TreeSet<String>(policy.getAllPrivileges(set("manager"), ActiveRoleSet.ALL))
         .toString());
   }
 
@@ -99,7 +99,7 @@ public abstract class AbstractTestSimplePolicyEngine {
         PERM_SERVER1_CUSTOMERS_SELECT, PERM_SERVER1_ANALYST_ALL,
         PERM_SERVER1_JUNIOR_ANALYST_READ));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPrivileges(set("analyst"), ActiveRoleSet.ALL))
+        new TreeSet<String>(policy.getAllPrivileges(set("analyst"), ActiveRoleSet.ALL))
         .toString());
   }
 
@@ -109,7 +109,7 @@ public abstract class AbstractTestSimplePolicyEngine {
         .newHashSet(PERM_SERVER1_JUNIOR_ANALYST_ALL,
             PERM_SERVER1_CUSTOMERS_DB_CUSTOMERS_PARTIAL_SELECT));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPrivileges(set("jranalyst"), ActiveRoleSet.ALL))
+        new TreeSet<String>(policy.getAllPrivileges(set("jranalyst"), ActiveRoleSet.ALL))
         .toString());
   }
 
@@ -117,7 +117,7 @@ public abstract class AbstractTestSimplePolicyEngine {
   public void testAdmin() throws Exception {
     Set<String> expected = Sets.newTreeSet(Sets.newHashSet(PERM_SERVER1_ADMIN));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPrivileges(set("admin"), ActiveRoleSet.ALL))
+        new TreeSet<String>(policy.getAllPrivileges(set("admin"), ActiveRoleSet.ALL))
         .toString());
   }
 
@@ -127,7 +127,7 @@ public abstract class AbstractTestSimplePolicyEngine {
     Set<String> expected = Sets.newTreeSet(Sets.newHashSet(
         PERM_SERVER1_OTHER_GROUP_DB_CUSTOMERS_SELECT));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPrivileges(set("other_group"), ActiveRoleSet.ALL))
+        new TreeSet<String>(policy.getAllPrivileges(set("other_group"), ActiveRoleSet.ALL))
         .toString());
   }
 
@@ -137,7 +137,7 @@ public abstract class AbstractTestSimplePolicyEngine {
         .newHashSet(PERM_SERVER1_JUNIOR_ANALYST_ALL,
             PERM_SERVER1_CUSTOMERS_DB_CUSTOMERS_PARTIAL_SELECT));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPrivileges(set("jranalyst"), ActiveRoleSet.ALL))
+        new TreeSet<String>(policy.getAllPrivileges(set("jranalyst"), ActiveRoleSet.ALL))
         .toString());
   }
 
@@ -146,7 +146,7 @@ public abstract class AbstractTestSimplePolicyEngine {
     Set<String> expected = Sets.newTreeSet(Sets.newHashSet(
         PERM_SERVER1_OTHER_GROUP_DB_CUSTOMERS_SELECT));
     Assert.assertEquals(expected.toString(),
-        new TreeSet<String>(policy.getPrivileges(set("other_group"), ActiveRoleSet.ALL))
+        new TreeSet<String>(policy.getAllPrivileges(set("other_group"), ActiveRoleSet.ALL))
         .toString());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestPolicyParsingNegative.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestPolicyParsingNegative.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestPolicyParsingNegative.java
index e88ae32..5f7c671 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestPolicyParsingNegative.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestPolicyParsingNegative.java
@@ -73,7 +73,7 @@ public class TestPolicyParsingNegative {
     append("[roles]", otherPolicyFile);
     append("malicious_role = server=server1->db=customers->table=purchases->action=select", otherPolicyFile);
     PolicyEngine policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
-    ImmutableSet<String> permissions = policy.getPrivileges(Sets.newHashSet("other_group"), ActiveRoleSet.ALL);
+    ImmutableSet<String> permissions = policy.getAllPrivileges(Sets.newHashSet("other_group"), ActiveRoleSet.ALL);
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
   @Test
@@ -87,21 +87,21 @@ public class TestPolicyParsingNegative {
     policyFile.write(globalPolicyFile);
     policyFile.write(otherPolicyFile);
     policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
-    permissions = policy.getPrivileges(Sets.newHashSet("admin"), ActiveRoleSet.ALL);
+    permissions = policy.getAllPrivileges(Sets.newHashSet("admin"), ActiveRoleSet.ALL);
     Assert.assertEquals(permissions.toString(), "[server=server1]");
     // test to ensure [users] fails parsing of per-db file
     policyFile.addDatabase("other", otherPolicyFile.getPath());
     policyFile.write(globalPolicyFile);
     policyFile.write(otherPolicyFile);
     policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
-    permissions = policy.getPrivileges(Sets.newHashSet("admin"), ActiveRoleSet.ALL);
+    permissions = policy.getAllPrivileges(Sets.newHashSet("admin"), ActiveRoleSet.ALL);
     Assert.assertEquals(permissions.toString(), "[server=server1]");
     // test to ensure [databases] fails parsing of per-db file
     // by removing the user mapping from the per-db policy file
     policyFile.removeGroupsFromUser("admin1", "admin")
       .write(otherPolicyFile);
     policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
-    permissions = policy.getPrivileges(Sets.newHashSet("admin"), ActiveRoleSet.ALL);
+    permissions = policy.getAllPrivileges(Sets.newHashSet("admin"), ActiveRoleSet.ALL);
     Assert.assertEquals(permissions.toString(), "[server=server1]");
   }
 
@@ -114,7 +114,7 @@ public class TestPolicyParsingNegative {
     append("[roles]", otherPolicyFile);
     append("malicious_role = server=server1", otherPolicyFile);
     PolicyEngine policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
-    ImmutableSet<String> permissions = policy.getPrivileges(Sets.newHashSet("other_group"), ActiveRoleSet.ALL);
+    ImmutableSet<String> permissions = policy.getAllPrivileges(Sets.newHashSet("other_group"), ActiveRoleSet.ALL);
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
 
@@ -125,7 +125,7 @@ public class TestPolicyParsingNegative {
     append("[roles]", globalPolicyFile);
     append("malicious_role = server=*", globalPolicyFile);
     PolicyEngine policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
-    ImmutableSet<String> permissions = policy.getPrivileges(Sets.newHashSet("group"), ActiveRoleSet.ALL);
+    ImmutableSet<String> permissions = policy.getAllPrivileges(Sets.newHashSet("group"), ActiveRoleSet.ALL);
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
 
@@ -136,7 +136,7 @@ public class TestPolicyParsingNegative {
     append("[roles]", globalPolicyFile);
     append("malicious_role = server=server2", globalPolicyFile);
     PolicyEngine policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
-    ImmutableSet<String> permissions = policy.getPrivileges(Sets.newHashSet("group"), ActiveRoleSet.ALL);
+    ImmutableSet<String> permissions = policy.getAllPrivileges(Sets.newHashSet("group"), ActiveRoleSet.ALL);
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
 
@@ -147,7 +147,7 @@ public class TestPolicyParsingNegative {
     append("[roles]", globalPolicyFile);
     append("malicious_role = *", globalPolicyFile);
     PolicyEngine policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
-    ImmutableSet<String> permissions = policy.getPrivileges(Sets.newHashSet("group"), ActiveRoleSet.ALL);
+    ImmutableSet<String> permissions = policy.getAllPrivileges(Sets.newHashSet("group"), ActiveRoleSet.ALL);
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
   }
 
@@ -185,10 +185,10 @@ public class TestPolicyParsingNegative {
     PolicyEngine policy = new DBPolicyFileBackend("server1", globalPolicyFile.getPath());
 
     // verify that the db1 rule is empty
-    ImmutableSet<String> permissions = policy.getPrivileges(Sets.newHashSet("db1_group"), ActiveRoleSet.ALL);
+    ImmutableSet<String> permissions = policy.getAllPrivileges(Sets.newHashSet("db1_group"), ActiveRoleSet.ALL);
     Assert.assertTrue(permissions.toString(), permissions.isEmpty());
 
-    permissions = policy.getPrivileges(Sets.newHashSet("db2_group"), ActiveRoleSet.ALL);
+    permissions = policy.getAllPrivileges(Sets.newHashSet("db2_group"), ActiveRoleSet.ALL);
     Assert.assertEquals(permissions.toString(), 1, permissions.size());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineDFS.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineDFS.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineDFS.java
index 08f84a3..f8c36e2 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineDFS.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestSimpleDBPolicyEngineDFS.java
@@ -108,7 +108,7 @@ public class TestSimpleDBPolicyEngineDFS extends AbstractTestSimplePolicyEngine
     Set<String> dbGroups = Sets.newHashSet();
     dbGroups.add("group1");
     ImmutableSet<String> dbPerms =
-        multiFSEngine.getPrivileges(dbGroups, ActiveRoleSet.ALL);
+        multiFSEngine.getAllPrivileges(dbGroups, ActiveRoleSet.ALL);
     Assert.assertEquals("No DB permissions found", 1, dbPerms.size());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SimpleSearchPolicyEngine.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SimpleSearchPolicyEngine.java b/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SimpleSearchPolicyEngine.java
index 8adcb6f..f428aea 100644
--- a/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SimpleSearchPolicyEngine.java
+++ b/sentry-policy/sentry-policy-search/src/main/java/org/apache/sentry/policy/search/SimpleSearchPolicyEngine.java
@@ -19,6 +19,7 @@ package org.apache.sentry.policy.search;
 import java.util.Set;
 
 import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.SentryConfigurationException;
 import org.apache.sentry.policy.common.PrivilegeFactory;
 import org.apache.sentry.policy.common.PolicyEngine;
@@ -61,7 +62,16 @@ public class SimpleSearchPolicyEngine implements PolicyEngine {
    * {@inheritDoc}
    */
   @Override
-  public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet) {
+  public ImmutableSet<String> getAllPrivileges(Set<String> groups,
+      ActiveRoleSet roleSet) throws SentryConfigurationException {
+    return getPrivileges(groups, roleSet);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet, Authorizable... authorizationHierarchy ) {
     if(LOGGER.isDebugEnabled()) {
       LOGGER.debug("Getting permissions for {}", groups);
     }
@@ -88,4 +98,5 @@ public class SimpleSearchPolicyEngine implements PolicyEngine {
       providerBackend.close();
     }
   }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-provider/sentry-provider-cache/src/main/java/org/apache/sentry/provider/cache/SimpleCacheProviderBackend.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-cache/src/main/java/org/apache/sentry/provider/cache/SimpleCacheProviderBackend.java b/sentry-provider/sentry-provider-cache/src/main/java/org/apache/sentry/provider/cache/SimpleCacheProviderBackend.java
index 1b0aba6..396c21c 100644
--- a/sentry-provider/sentry-provider-cache/src/main/java/org/apache/sentry/provider/cache/SimpleCacheProviderBackend.java
+++ b/sentry-provider/sentry-provider-cache/src/main/java/org/apache/sentry/provider/cache/SimpleCacheProviderBackend.java
@@ -21,6 +21,7 @@ import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.SentryConfigurationException;
 import org.apache.sentry.provider.common.ProviderBackend;
 import org.apache.sentry.provider.common.ProviderBackendContext;
@@ -48,7 +49,7 @@ public class SimpleCacheProviderBackend implements ProviderBackend {
 
   @Override
   public ImmutableSet<String> getPrivileges(Set<String> groups,
-      ActiveRoleSet roleSet) {
+      ActiveRoleSet roleSet, Authorizable... authorizationhierarchy) {
     if (!initialized()) {
       throw new IllegalStateException(
           "Backend has not been properly initialized");

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderBackend.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderBackend.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderBackend.java
index a175245..ddb9cf9 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderBackend.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderBackend.java
@@ -21,6 +21,7 @@ import java.util.Set;
 import javax.annotation.concurrent.ThreadSafe;
 
 import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.SentryConfigurationException;
 
 import com.google.common.collect.ImmutableSet;
@@ -46,7 +47,7 @@ public interface ProviderBackend {
   /**
    * Get the privileges from the backend.
    */
-  public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet);
+  public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet, Authorizable... authorizableHierarchy);
 
   /**
    * Get the roles associated with the groups from the backend.

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
index e1e7f4a..3a993b0 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
@@ -95,7 +95,7 @@ public abstract class ResourceAuthorizationProvider implements AuthorizationProv
     for (Authorizable authorizable : authorizables) {
       hierarchy.add(KV_JOINER.join(authorizable.getTypeName(), authorizable.getName()));
     }
-    Iterable<Privilege> privileges = getPrivileges(groups, roleSet);
+    Iterable<Privilege> privileges = getPrivileges(groups, roleSet, authorizables.toArray(new Authorizable[0]));
     List<String> requestPrivileges = buildPermissions(authorizables, actions);
     lastFailedPrivileges.get().clear();
 
@@ -118,8 +118,8 @@ public abstract class ResourceAuthorizationProvider implements AuthorizationProv
     return false;
   }
 
-  private Iterable<Privilege> getPrivileges(Set<String> groups, ActiveRoleSet roleSet) {
-    return Iterables.transform(policy.getPrivileges(groups, roleSet),
+  private Iterable<Privilege> getPrivileges(Set<String> groups, ActiveRoleSet roleSet, Authorizable[] authorizables) {
+    return Iterables.transform(policy.getPrivileges(groups, roleSet, authorizables),
         new Function<String, Privilege>() {
       @Override
       public Privilege apply(String privilege) {
@@ -144,12 +144,12 @@ public abstract class ResourceAuthorizationProvider implements AuthorizationProv
 
   @Override
   public Set<String> listPrivilegesForSubject(Subject subject) throws SentryConfigurationException {
-    return policy.getPrivileges(getGroups(subject), ActiveRoleSet.ALL);
+    return policy.getPrivileges(getGroups(subject), ActiveRoleSet.ALL, null);
   }
 
   @Override
   public Set<String> listPrivilegesForGroup(String groupName) throws SentryConfigurationException {
-    return policy.getPrivileges(Sets.newHashSet(groupName), ActiveRoleSet.ALL);
+    return policy.getPrivileges(Sets.newHashSet(groupName), ActiveRoleSet.ALL, null);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/TestGetGroupMapping.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/TestGetGroupMapping.java b/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/TestGetGroupMapping.java
index ece740b..f57198a 100644
--- a/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/TestGetGroupMapping.java
+++ b/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/TestGetGroupMapping.java
@@ -19,7 +19,9 @@ package org.apache.sentry.provider.common;
 import static org.junit.Assert.assertSame;
 
 import java.util.Set;
+import java.util.List;
 
+import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.SentryConfigurationException;
 import org.apache.sentry.core.common.ActiveRoleSet;
 import org.apache.sentry.policy.common.PrivilegeFactory;
@@ -44,15 +46,25 @@ public class TestGetGroupMapping {
   public void testResourceAuthorizationProvider() {
     final Set<String> set = Sets.newHashSet("a", "b", "c");
     GroupMappingService mappingService = new GroupMappingService() {
+      @Override
       public Set<String> getGroups(String user) { return set; }
     };
     PolicyEngine policyEngine = new PolicyEngine() {
+      @Override
       public PrivilegeFactory getPrivilegeFactory() { return null; }
 
-      public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet) {
+      @Override
+      public ImmutableSet<String> getAllPrivileges(Set<String> groups,
+          ActiveRoleSet roleSet) throws SentryConfigurationException {
+        return getPrivileges(groups, roleSet, null);
+      }
+
+      @Override
+      public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet, Authorizable... authorizableHierarchy) {
         return ImmutableSet.of();
       }
 
+      @Override
       public void validatePolicy(boolean strictValidation)
           throws SentryConfigurationException {
         return;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryPrivilegesForProviderRequest.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryPrivilegesForProviderRequest.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryPrivilegesForProviderRequest.java
index 65fd5b5..10ab56b 100644
--- a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryPrivilegesForProviderRequest.java
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryPrivilegesForProviderRequest.java
@@ -37,6 +37,7 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
   private static final org.apache.thrift.protocol.TField PROTOCOL_VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("protocol_version", org.apache.thrift.protocol.TType.I32, (short)1);
   private static final org.apache.thrift.protocol.TField GROUPS_FIELD_DESC = new org.apache.thrift.protocol.TField("groups", org.apache.thrift.protocol.TType.SET, (short)2);
   private static final org.apache.thrift.protocol.TField ROLE_SET_FIELD_DESC = new org.apache.thrift.protocol.TField("roleSet", org.apache.thrift.protocol.TType.STRUCT, (short)3);
+  private static final org.apache.thrift.protocol.TField AUTHORIZABLE_HIERARCHY_FIELD_DESC = new org.apache.thrift.protocol.TField("authorizableHierarchy", org.apache.thrift.protocol.TType.STRUCT, (short)4);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -47,12 +48,14 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
   private int protocol_version; // required
   private Set<String> groups; // required
   private TSentryActiveRoleSet roleSet; // required
+  private TSentryAuthorizable authorizableHierarchy; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     PROTOCOL_VERSION((short)1, "protocol_version"),
     GROUPS((short)2, "groups"),
-    ROLE_SET((short)3, "roleSet");
+    ROLE_SET((short)3, "roleSet"),
+    AUTHORIZABLE_HIERARCHY((short)4, "authorizableHierarchy");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -73,6 +76,8 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
           return GROUPS;
         case 3: // ROLE_SET
           return ROLE_SET;
+        case 4: // AUTHORIZABLE_HIERARCHY
+          return AUTHORIZABLE_HIERARCHY;
         default:
           return null;
       }
@@ -115,6 +120,7 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
   // isset id assignments
   private static final int __PROTOCOL_VERSION_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
+  private _Fields optionals[] = {_Fields.AUTHORIZABLE_HIERARCHY};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -125,6 +131,8 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
             new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
     tmpMap.put(_Fields.ROLE_SET, new org.apache.thrift.meta_data.FieldMetaData("roleSet", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSentryActiveRoleSet.class)));
+    tmpMap.put(_Fields.AUTHORIZABLE_HIERARCHY, new org.apache.thrift.meta_data.FieldMetaData("authorizableHierarchy", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSentryAuthorizable.class)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TListSentryPrivilegesForProviderRequest.class, metaDataMap);
   }
@@ -162,6 +170,9 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
     if (other.isSetRoleSet()) {
       this.roleSet = new TSentryActiveRoleSet(other.roleSet);
     }
+    if (other.isSetAuthorizableHierarchy()) {
+      this.authorizableHierarchy = new TSentryAuthorizable(other.authorizableHierarchy);
+    }
   }
 
   public TListSentryPrivilegesForProviderRequest deepCopy() {
@@ -174,6 +185,7 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
 
     this.groups = null;
     this.roleSet = null;
+    this.authorizableHierarchy = null;
   }
 
   public int getProtocol_version() {
@@ -259,6 +271,29 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
     }
   }
 
+  public TSentryAuthorizable getAuthorizableHierarchy() {
+    return this.authorizableHierarchy;
+  }
+
+  public void setAuthorizableHierarchy(TSentryAuthorizable authorizableHierarchy) {
+    this.authorizableHierarchy = authorizableHierarchy;
+  }
+
+  public void unsetAuthorizableHierarchy() {
+    this.authorizableHierarchy = null;
+  }
+
+  /** Returns true if field authorizableHierarchy is set (has been assigned a value) and false otherwise */
+  public boolean isSetAuthorizableHierarchy() {
+    return this.authorizableHierarchy != null;
+  }
+
+  public void setAuthorizableHierarchyIsSet(boolean value) {
+    if (!value) {
+      this.authorizableHierarchy = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case PROTOCOL_VERSION:
@@ -285,6 +320,14 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
       }
       break;
 
+    case AUTHORIZABLE_HIERARCHY:
+      if (value == null) {
+        unsetAuthorizableHierarchy();
+      } else {
+        setAuthorizableHierarchy((TSentryAuthorizable)value);
+      }
+      break;
+
     }
   }
 
@@ -299,6 +342,9 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
     case ROLE_SET:
       return getRoleSet();
 
+    case AUTHORIZABLE_HIERARCHY:
+      return getAuthorizableHierarchy();
+
     }
     throw new IllegalStateException();
   }
@@ -316,6 +362,8 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
       return isSetGroups();
     case ROLE_SET:
       return isSetRoleSet();
+    case AUTHORIZABLE_HIERARCHY:
+      return isSetAuthorizableHierarchy();
     }
     throw new IllegalStateException();
   }
@@ -360,6 +408,15 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
         return false;
     }
 
+    boolean this_present_authorizableHierarchy = true && this.isSetAuthorizableHierarchy();
+    boolean that_present_authorizableHierarchy = true && that.isSetAuthorizableHierarchy();
+    if (this_present_authorizableHierarchy || that_present_authorizableHierarchy) {
+      if (!(this_present_authorizableHierarchy && that_present_authorizableHierarchy))
+        return false;
+      if (!this.authorizableHierarchy.equals(that.authorizableHierarchy))
+        return false;
+    }
+
     return true;
   }
 
@@ -382,6 +439,11 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
     if (present_roleSet)
       builder.append(roleSet);
 
+    boolean present_authorizableHierarchy = true && (isSetAuthorizableHierarchy());
+    builder.append(present_authorizableHierarchy);
+    if (present_authorizableHierarchy)
+      builder.append(authorizableHierarchy);
+
     return builder.toHashCode();
   }
 
@@ -423,6 +485,16 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetAuthorizableHierarchy()).compareTo(typedOther.isSetAuthorizableHierarchy());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetAuthorizableHierarchy()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.authorizableHierarchy, typedOther.authorizableHierarchy);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -462,6 +534,16 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
       sb.append(this.roleSet);
     }
     first = false;
+    if (isSetAuthorizableHierarchy()) {
+      if (!first) sb.append(", ");
+      sb.append("authorizableHierarchy:");
+      if (this.authorizableHierarchy == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.authorizableHierarchy);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -484,6 +566,9 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
     if (roleSet != null) {
       roleSet.validate();
     }
+    if (authorizableHierarchy != null) {
+      authorizableHierarchy.validate();
+    }
   }
 
   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -557,6 +642,15 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 4: // AUTHORIZABLE_HIERARCHY
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.authorizableHierarchy = new TSentryAuthorizable();
+              struct.authorizableHierarchy.read(iprot);
+              struct.setAuthorizableHierarchyIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -590,6 +684,13 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
         struct.roleSet.write(oprot);
         oprot.writeFieldEnd();
       }
+      if (struct.authorizableHierarchy != null) {
+        if (struct.isSetAuthorizableHierarchy()) {
+          oprot.writeFieldBegin(AUTHORIZABLE_HIERARCHY_FIELD_DESC);
+          struct.authorizableHierarchy.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -616,6 +717,14 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
         }
       }
       struct.roleSet.write(oprot);
+      BitSet optionals = new BitSet();
+      if (struct.isSetAuthorizableHierarchy()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetAuthorizableHierarchy()) {
+        struct.authorizableHierarchy.write(oprot);
+      }
     }
 
     @Override
@@ -637,6 +746,12 @@ public class TListSentryPrivilegesForProviderRequest implements org.apache.thrif
       struct.roleSet = new TSentryActiveRoleSet();
       struct.roleSet.read(iprot);
       struct.setRoleSetIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.authorizableHierarchy = new TSentryAuthorizable();
+        struct.authorizableHierarchy.read(iprot);
+        struct.setAuthorizableHierarchyIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryPrivilegesRequest.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryPrivilegesRequest.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryPrivilegesRequest.java
index 89afb70..393ff91 100644
--- a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryPrivilegesRequest.java
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TListSentryPrivilegesRequest.java
@@ -36,7 +36,8 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
 
   private static final org.apache.thrift.protocol.TField PROTOCOL_VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("protocol_version", org.apache.thrift.protocol.TType.I32, (short)1);
   private static final org.apache.thrift.protocol.TField REQUESTOR_USER_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("requestorUserName", org.apache.thrift.protocol.TType.STRING, (short)2);
-  private static final org.apache.thrift.protocol.TField ROLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("roleName", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField ROLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("roleName", org.apache.thrift.protocol.TType.STRING, (short)4);
+  private static final org.apache.thrift.protocol.TField AUTHORIZABLE_HIERARCHY_FIELD_DESC = new org.apache.thrift.protocol.TField("authorizableHierarchy", org.apache.thrift.protocol.TType.STRUCT, (short)5);
 
   private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
   static {
@@ -47,12 +48,14 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
   private int protocol_version; // required
   private String requestorUserName; // required
   private String roleName; // required
+  private TSentryAuthorizable authorizableHierarchy; // optional
 
   /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
   public enum _Fields implements org.apache.thrift.TFieldIdEnum {
     PROTOCOL_VERSION((short)1, "protocol_version"),
     REQUESTOR_USER_NAME((short)2, "requestorUserName"),
-    ROLE_NAME((short)3, "roleName");
+    ROLE_NAME((short)4, "roleName"),
+    AUTHORIZABLE_HIERARCHY((short)5, "authorizableHierarchy");
 
     private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
 
@@ -71,8 +74,10 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
           return PROTOCOL_VERSION;
         case 2: // REQUESTOR_USER_NAME
           return REQUESTOR_USER_NAME;
-        case 3: // ROLE_NAME
+        case 4: // ROLE_NAME
           return ROLE_NAME;
+        case 5: // AUTHORIZABLE_HIERARCHY
+          return AUTHORIZABLE_HIERARCHY;
         default:
           return null;
       }
@@ -115,6 +120,7 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
   // isset id assignments
   private static final int __PROTOCOL_VERSION_ISSET_ID = 0;
   private byte __isset_bitfield = 0;
+  private _Fields optionals[] = {_Fields.AUTHORIZABLE_HIERARCHY};
   public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
   static {
     Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
@@ -124,6 +130,8 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
     tmpMap.put(_Fields.ROLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("roleName", org.apache.thrift.TFieldRequirementType.REQUIRED, 
         new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.AUTHORIZABLE_HIERARCHY, new org.apache.thrift.meta_data.FieldMetaData("authorizableHierarchy", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSentryAuthorizable.class)));
     metaDataMap = Collections.unmodifiableMap(tmpMap);
     org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TListSentryPrivilegesRequest.class, metaDataMap);
   }
@@ -157,6 +165,9 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
     if (other.isSetRoleName()) {
       this.roleName = other.roleName;
     }
+    if (other.isSetAuthorizableHierarchy()) {
+      this.authorizableHierarchy = new TSentryAuthorizable(other.authorizableHierarchy);
+    }
   }
 
   public TListSentryPrivilegesRequest deepCopy() {
@@ -169,6 +180,7 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
 
     this.requestorUserName = null;
     this.roleName = null;
+    this.authorizableHierarchy = null;
   }
 
   public int getProtocol_version() {
@@ -239,6 +251,29 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
     }
   }
 
+  public TSentryAuthorizable getAuthorizableHierarchy() {
+    return this.authorizableHierarchy;
+  }
+
+  public void setAuthorizableHierarchy(TSentryAuthorizable authorizableHierarchy) {
+    this.authorizableHierarchy = authorizableHierarchy;
+  }
+
+  public void unsetAuthorizableHierarchy() {
+    this.authorizableHierarchy = null;
+  }
+
+  /** Returns true if field authorizableHierarchy is set (has been assigned a value) and false otherwise */
+  public boolean isSetAuthorizableHierarchy() {
+    return this.authorizableHierarchy != null;
+  }
+
+  public void setAuthorizableHierarchyIsSet(boolean value) {
+    if (!value) {
+      this.authorizableHierarchy = null;
+    }
+  }
+
   public void setFieldValue(_Fields field, Object value) {
     switch (field) {
     case PROTOCOL_VERSION:
@@ -265,6 +300,14 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
       }
       break;
 
+    case AUTHORIZABLE_HIERARCHY:
+      if (value == null) {
+        unsetAuthorizableHierarchy();
+      } else {
+        setAuthorizableHierarchy((TSentryAuthorizable)value);
+      }
+      break;
+
     }
   }
 
@@ -279,6 +322,9 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
     case ROLE_NAME:
       return getRoleName();
 
+    case AUTHORIZABLE_HIERARCHY:
+      return getAuthorizableHierarchy();
+
     }
     throw new IllegalStateException();
   }
@@ -296,6 +342,8 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
       return isSetRequestorUserName();
     case ROLE_NAME:
       return isSetRoleName();
+    case AUTHORIZABLE_HIERARCHY:
+      return isSetAuthorizableHierarchy();
     }
     throw new IllegalStateException();
   }
@@ -340,6 +388,15 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
         return false;
     }
 
+    boolean this_present_authorizableHierarchy = true && this.isSetAuthorizableHierarchy();
+    boolean that_present_authorizableHierarchy = true && that.isSetAuthorizableHierarchy();
+    if (this_present_authorizableHierarchy || that_present_authorizableHierarchy) {
+      if (!(this_present_authorizableHierarchy && that_present_authorizableHierarchy))
+        return false;
+      if (!this.authorizableHierarchy.equals(that.authorizableHierarchy))
+        return false;
+    }
+
     return true;
   }
 
@@ -362,6 +419,11 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
     if (present_roleName)
       builder.append(roleName);
 
+    boolean present_authorizableHierarchy = true && (isSetAuthorizableHierarchy());
+    builder.append(present_authorizableHierarchy);
+    if (present_authorizableHierarchy)
+      builder.append(authorizableHierarchy);
+
     return builder.toHashCode();
   }
 
@@ -403,6 +465,16 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
         return lastComparison;
       }
     }
+    lastComparison = Boolean.valueOf(isSetAuthorizableHierarchy()).compareTo(typedOther.isSetAuthorizableHierarchy());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetAuthorizableHierarchy()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.authorizableHierarchy, typedOther.authorizableHierarchy);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
     return 0;
   }
 
@@ -442,6 +514,16 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
       sb.append(this.roleName);
     }
     first = false;
+    if (isSetAuthorizableHierarchy()) {
+      if (!first) sb.append(", ");
+      sb.append("authorizableHierarchy:");
+      if (this.authorizableHierarchy == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.authorizableHierarchy);
+      }
+      first = false;
+    }
     sb.append(")");
     return sb.toString();
   }
@@ -461,6 +543,9 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
     }
 
     // check for sub-struct validity
+    if (authorizableHierarchy != null) {
+      authorizableHierarchy.validate();
+    }
   }
 
   private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
@@ -515,7 +600,7 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
-          case 3: // ROLE_NAME
+          case 4: // ROLE_NAME
             if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
               struct.roleName = iprot.readString();
               struct.setRoleNameIsSet(true);
@@ -523,6 +608,15 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
               org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
             }
             break;
+          case 5: // AUTHORIZABLE_HIERARCHY
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+              struct.authorizableHierarchy = new TSentryAuthorizable();
+              struct.authorizableHierarchy.read(iprot);
+              struct.setAuthorizableHierarchyIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
           default:
             org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
         }
@@ -549,6 +643,13 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
         oprot.writeString(struct.roleName);
         oprot.writeFieldEnd();
       }
+      if (struct.authorizableHierarchy != null) {
+        if (struct.isSetAuthorizableHierarchy()) {
+          oprot.writeFieldBegin(AUTHORIZABLE_HIERARCHY_FIELD_DESC);
+          struct.authorizableHierarchy.write(oprot);
+          oprot.writeFieldEnd();
+        }
+      }
       oprot.writeFieldStop();
       oprot.writeStructEnd();
     }
@@ -569,6 +670,14 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
       oprot.writeI32(struct.protocol_version);
       oprot.writeString(struct.requestorUserName);
       oprot.writeString(struct.roleName);
+      BitSet optionals = new BitSet();
+      if (struct.isSetAuthorizableHierarchy()) {
+        optionals.set(0);
+      }
+      oprot.writeBitSet(optionals, 1);
+      if (struct.isSetAuthorizableHierarchy()) {
+        struct.authorizableHierarchy.write(oprot);
+      }
     }
 
     @Override
@@ -580,6 +689,12 @@ public class TListSentryPrivilegesRequest implements org.apache.thrift.TBase<TLi
       struct.setRequestorUserNameIsSet(true);
       struct.roleName = iprot.readString();
       struct.setRoleNameIsSet(true);
+      BitSet incoming = iprot.readBitSet(1);
+      if (incoming.get(0)) {
+        struct.authorizableHierarchy = new TSentryAuthorizable();
+        struct.authorizableHierarchy.read(iprot);
+        struct.setAuthorizableHierarchyIsSet(true);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TSentryAuthorizable.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TSentryAuthorizable.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TSentryAuthorizable.java
new file mode 100644
index 0000000..59418a3
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/TSentryAuthorizable.java
@@ -0,0 +1,707 @@
+/**
+ * Autogenerated by Thrift Compiler (0.9.0)
+ *
+ * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING
+ *  @generated
+ */
+package org.apache.sentry.provider.db.service.thrift;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.thrift.scheme.IScheme;
+import org.apache.thrift.scheme.SchemeFactory;
+import org.apache.thrift.scheme.StandardScheme;
+
+import org.apache.thrift.scheme.TupleScheme;
+import org.apache.thrift.protocol.TTupleProtocol;
+import org.apache.thrift.protocol.TProtocolException;
+import org.apache.thrift.EncodingUtils;
+import org.apache.thrift.TException;
+import java.util.List;
+import java.util.ArrayList;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.EnumMap;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.EnumSet;
+import java.util.Collections;
+import java.util.BitSet;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TSentryAuthorizable implements org.apache.thrift.TBase<TSentryAuthorizable, TSentryAuthorizable._Fields>, java.io.Serializable, Cloneable {
+  private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TSentryAuthorizable");
+
+  private static final org.apache.thrift.protocol.TField SERVER_FIELD_DESC = new org.apache.thrift.protocol.TField("server", org.apache.thrift.protocol.TType.STRING, (short)1);
+  private static final org.apache.thrift.protocol.TField URI_FIELD_DESC = new org.apache.thrift.protocol.TField("uri", org.apache.thrift.protocol.TType.STRING, (short)2);
+  private static final org.apache.thrift.protocol.TField DB_FIELD_DESC = new org.apache.thrift.protocol.TField("db", org.apache.thrift.protocol.TType.STRING, (short)3);
+  private static final org.apache.thrift.protocol.TField TABLE_FIELD_DESC = new org.apache.thrift.protocol.TField("table", org.apache.thrift.protocol.TType.STRING, (short)4);
+
+  private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+  static {
+    schemes.put(StandardScheme.class, new TSentryAuthorizableStandardSchemeFactory());
+    schemes.put(TupleScheme.class, new TSentryAuthorizableTupleSchemeFactory());
+  }
+
+  private String server; // required
+  private String uri; // optional
+  private String db; // optional
+  private String table; // optional
+
+  /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+  public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+    SERVER((short)1, "server"),
+    URI((short)2, "uri"),
+    DB((short)3, "db"),
+    TABLE((short)4, "table");
+
+    private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+    static {
+      for (_Fields field : EnumSet.allOf(_Fields.class)) {
+        byName.put(field.getFieldName(), field);
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, or null if its not found.
+     */
+    public static _Fields findByThriftId(int fieldId) {
+      switch(fieldId) {
+        case 1: // SERVER
+          return SERVER;
+        case 2: // URI
+          return URI;
+        case 3: // DB
+          return DB;
+        case 4: // TABLE
+          return TABLE;
+        default:
+          return null;
+      }
+    }
+
+    /**
+     * Find the _Fields constant that matches fieldId, throwing an exception
+     * if it is not found.
+     */
+    public static _Fields findByThriftIdOrThrow(int fieldId) {
+      _Fields fields = findByThriftId(fieldId);
+      if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+      return fields;
+    }
+
+    /**
+     * Find the _Fields constant that matches name, or null if its not found.
+     */
+    public static _Fields findByName(String name) {
+      return byName.get(name);
+    }
+
+    private final short _thriftId;
+    private final String _fieldName;
+
+    _Fields(short thriftId, String fieldName) {
+      _thriftId = thriftId;
+      _fieldName = fieldName;
+    }
+
+    public short getThriftFieldId() {
+      return _thriftId;
+    }
+
+    public String getFieldName() {
+      return _fieldName;
+    }
+  }
+
+  // isset id assignments
+  private _Fields optionals[] = {_Fields.URI,_Fields.DB,_Fields.TABLE};
+  public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+  static {
+    Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+    tmpMap.put(_Fields.SERVER, new org.apache.thrift.meta_data.FieldMetaData("server", org.apache.thrift.TFieldRequirementType.REQUIRED, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.URI, new org.apache.thrift.meta_data.FieldMetaData("uri", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.DB, new org.apache.thrift.meta_data.FieldMetaData("db", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    tmpMap.put(_Fields.TABLE, new org.apache.thrift.meta_data.FieldMetaData("table", org.apache.thrift.TFieldRequirementType.OPTIONAL, 
+        new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)));
+    metaDataMap = Collections.unmodifiableMap(tmpMap);
+    org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TSentryAuthorizable.class, metaDataMap);
+  }
+
+  public TSentryAuthorizable() {
+  }
+
+  public TSentryAuthorizable(
+    String server)
+  {
+    this();
+    this.server = server;
+  }
+
+  /**
+   * Performs a deep copy on <i>other</i>.
+   */
+  public TSentryAuthorizable(TSentryAuthorizable other) {
+    if (other.isSetServer()) {
+      this.server = other.server;
+    }
+    if (other.isSetUri()) {
+      this.uri = other.uri;
+    }
+    if (other.isSetDb()) {
+      this.db = other.db;
+    }
+    if (other.isSetTable()) {
+      this.table = other.table;
+    }
+  }
+
+  public TSentryAuthorizable deepCopy() {
+    return new TSentryAuthorizable(this);
+  }
+
+  @Override
+  public void clear() {
+    this.server = null;
+    this.uri = null;
+    this.db = null;
+    this.table = null;
+  }
+
+  public String getServer() {
+    return this.server;
+  }
+
+  public void setServer(String server) {
+    this.server = server;
+  }
+
+  public void unsetServer() {
+    this.server = null;
+  }
+
+  /** Returns true if field server is set (has been assigned a value) and false otherwise */
+  public boolean isSetServer() {
+    return this.server != null;
+  }
+
+  public void setServerIsSet(boolean value) {
+    if (!value) {
+      this.server = null;
+    }
+  }
+
+  public String getUri() {
+    return this.uri;
+  }
+
+  public void setUri(String uri) {
+    this.uri = uri;
+  }
+
+  public void unsetUri() {
+    this.uri = null;
+  }
+
+  /** Returns true if field uri is set (has been assigned a value) and false otherwise */
+  public boolean isSetUri() {
+    return this.uri != null;
+  }
+
+  public void setUriIsSet(boolean value) {
+    if (!value) {
+      this.uri = null;
+    }
+  }
+
+  public String getDb() {
+    return this.db;
+  }
+
+  public void setDb(String db) {
+    this.db = db;
+  }
+
+  public void unsetDb() {
+    this.db = null;
+  }
+
+  /** Returns true if field db is set (has been assigned a value) and false otherwise */
+  public boolean isSetDb() {
+    return this.db != null;
+  }
+
+  public void setDbIsSet(boolean value) {
+    if (!value) {
+      this.db = null;
+    }
+  }
+
+  public String getTable() {
+    return this.table;
+  }
+
+  public void setTable(String table) {
+    this.table = table;
+  }
+
+  public void unsetTable() {
+    this.table = null;
+  }
+
+  /** Returns true if field table is set (has been assigned a value) and false otherwise */
+  public boolean isSetTable() {
+    return this.table != null;
+  }
+
+  public void setTableIsSet(boolean value) {
+    if (!value) {
+      this.table = null;
+    }
+  }
+
+  public void setFieldValue(_Fields field, Object value) {
+    switch (field) {
+    case SERVER:
+      if (value == null) {
+        unsetServer();
+      } else {
+        setServer((String)value);
+      }
+      break;
+
+    case URI:
+      if (value == null) {
+        unsetUri();
+      } else {
+        setUri((String)value);
+      }
+      break;
+
+    case DB:
+      if (value == null) {
+        unsetDb();
+      } else {
+        setDb((String)value);
+      }
+      break;
+
+    case TABLE:
+      if (value == null) {
+        unsetTable();
+      } else {
+        setTable((String)value);
+      }
+      break;
+
+    }
+  }
+
+  public Object getFieldValue(_Fields field) {
+    switch (field) {
+    case SERVER:
+      return getServer();
+
+    case URI:
+      return getUri();
+
+    case DB:
+      return getDb();
+
+    case TABLE:
+      return getTable();
+
+    }
+    throw new IllegalStateException();
+  }
+
+  /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+  public boolean isSet(_Fields field) {
+    if (field == null) {
+      throw new IllegalArgumentException();
+    }
+
+    switch (field) {
+    case SERVER:
+      return isSetServer();
+    case URI:
+      return isSetUri();
+    case DB:
+      return isSetDb();
+    case TABLE:
+      return isSetTable();
+    }
+    throw new IllegalStateException();
+  }
+
+  @Override
+  public boolean equals(Object that) {
+    if (that == null)
+      return false;
+    if (that instanceof TSentryAuthorizable)
+      return this.equals((TSentryAuthorizable)that);
+    return false;
+  }
+
+  public boolean equals(TSentryAuthorizable that) {
+    if (that == null)
+      return false;
+
+    boolean this_present_server = true && this.isSetServer();
+    boolean that_present_server = true && that.isSetServer();
+    if (this_present_server || that_present_server) {
+      if (!(this_present_server && that_present_server))
+        return false;
+      if (!this.server.equals(that.server))
+        return false;
+    }
+
+    boolean this_present_uri = true && this.isSetUri();
+    boolean that_present_uri = true && that.isSetUri();
+    if (this_present_uri || that_present_uri) {
+      if (!(this_present_uri && that_present_uri))
+        return false;
+      if (!this.uri.equals(that.uri))
+        return false;
+    }
+
+    boolean this_present_db = true && this.isSetDb();
+    boolean that_present_db = true && that.isSetDb();
+    if (this_present_db || that_present_db) {
+      if (!(this_present_db && that_present_db))
+        return false;
+      if (!this.db.equals(that.db))
+        return false;
+    }
+
+    boolean this_present_table = true && this.isSetTable();
+    boolean that_present_table = true && that.isSetTable();
+    if (this_present_table || that_present_table) {
+      if (!(this_present_table && that_present_table))
+        return false;
+      if (!this.table.equals(that.table))
+        return false;
+    }
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    HashCodeBuilder builder = new HashCodeBuilder();
+
+    boolean present_server = true && (isSetServer());
+    builder.append(present_server);
+    if (present_server)
+      builder.append(server);
+
+    boolean present_uri = true && (isSetUri());
+    builder.append(present_uri);
+    if (present_uri)
+      builder.append(uri);
+
+    boolean present_db = true && (isSetDb());
+    builder.append(present_db);
+    if (present_db)
+      builder.append(db);
+
+    boolean present_table = true && (isSetTable());
+    builder.append(present_table);
+    if (present_table)
+      builder.append(table);
+
+    return builder.toHashCode();
+  }
+
+  public int compareTo(TSentryAuthorizable other) {
+    if (!getClass().equals(other.getClass())) {
+      return getClass().getName().compareTo(other.getClass().getName());
+    }
+
+    int lastComparison = 0;
+    TSentryAuthorizable typedOther = (TSentryAuthorizable)other;
+
+    lastComparison = Boolean.valueOf(isSetServer()).compareTo(typedOther.isSetServer());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetServer()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.server, typedOther.server);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetUri()).compareTo(typedOther.isSetUri());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetUri()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.uri, typedOther.uri);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetDb()).compareTo(typedOther.isSetDb());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetDb()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.db, typedOther.db);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    lastComparison = Boolean.valueOf(isSetTable()).compareTo(typedOther.isSetTable());
+    if (lastComparison != 0) {
+      return lastComparison;
+    }
+    if (isSetTable()) {
+      lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.table, typedOther.table);
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+    }
+    return 0;
+  }
+
+  public _Fields fieldForId(int fieldId) {
+    return _Fields.findByThriftId(fieldId);
+  }
+
+  public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+    schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+  }
+
+  public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+    schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("TSentryAuthorizable(");
+    boolean first = true;
+
+    sb.append("server:");
+    if (this.server == null) {
+      sb.append("null");
+    } else {
+      sb.append(this.server);
+    }
+    first = false;
+    if (isSetUri()) {
+      if (!first) sb.append(", ");
+      sb.append("uri:");
+      if (this.uri == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.uri);
+      }
+      first = false;
+    }
+    if (isSetDb()) {
+      if (!first) sb.append(", ");
+      sb.append("db:");
+      if (this.db == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.db);
+      }
+      first = false;
+    }
+    if (isSetTable()) {
+      if (!first) sb.append(", ");
+      sb.append("table:");
+      if (this.table == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.table);
+      }
+      first = false;
+    }
+    sb.append(")");
+    return sb.toString();
+  }
+
+  public void validate() throws org.apache.thrift.TException {
+    // check for required fields
+    if (!isSetServer()) {
+      throw new org.apache.thrift.protocol.TProtocolException("Required field 'server' is unset! Struct:" + toString());
+    }
+
+    // check for sub-struct validity
+  }
+
+  private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+    try {
+      write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+    try {
+      read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+    } catch (org.apache.thrift.TException te) {
+      throw new java.io.IOException(te);
+    }
+  }
+
+  private static class TSentryAuthorizableStandardSchemeFactory implements SchemeFactory {
+    public TSentryAuthorizableStandardScheme getScheme() {
+      return new TSentryAuthorizableStandardScheme();
+    }
+  }
+
+  private static class TSentryAuthorizableStandardScheme extends StandardScheme<TSentryAuthorizable> {
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot, TSentryAuthorizable struct) throws org.apache.thrift.TException {
+      org.apache.thrift.protocol.TField schemeField;
+      iprot.readStructBegin();
+      while (true)
+      {
+        schemeField = iprot.readFieldBegin();
+        if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+          break;
+        }
+        switch (schemeField.id) {
+          case 1: // SERVER
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.server = iprot.readString();
+              struct.setServerIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 2: // URI
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.uri = iprot.readString();
+              struct.setUriIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 3: // DB
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.db = iprot.readString();
+              struct.setDbIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          case 4: // TABLE
+            if (schemeField.type == org.apache.thrift.protocol.TType.STRING) {
+              struct.table = iprot.readString();
+              struct.setTableIsSet(true);
+            } else { 
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+            }
+            break;
+          default:
+            org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+        }
+        iprot.readFieldEnd();
+      }
+      iprot.readStructEnd();
+      struct.validate();
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot, TSentryAuthorizable struct) throws org.apache.thrift.TException {
+      struct.validate();
+
+      oprot.writeStructBegin(STRUCT_DESC);
+      if (struct.server != null) {
+        oprot.writeFieldBegin(SERVER_FIELD_DESC);
+        oprot.writeString(struct.server);
+        oprot.writeFieldEnd();
+      }
+      if (struct.uri != null) {
+        if (struct.isSetUri()) {
+          oprot.writeFieldBegin(URI_FIELD_DESC);
+          oprot.writeString(struct.uri);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.db != null) {
+        if (struct.isSetDb()) {
+          oprot.writeFieldBegin(DB_FIELD_DESC);
+          oprot.writeString(struct.db);
+          oprot.writeFieldEnd();
+        }
+      }
+      if (struct.table != null) {
+        if (struct.isSetTable()) {
+          oprot.writeFieldBegin(TABLE_FIELD_DESC);
+          oprot.writeString(struct.table);
+          oprot.writeFieldEnd();
+        }
+      }
+      oprot.writeFieldStop();
+      oprot.writeStructEnd();
+    }
+
+  }
+
+  private static class TSentryAuthorizableTupleSchemeFactory implements SchemeFactory {
+    public TSentryAuthorizableTupleScheme getScheme() {
+      return new TSentryAuthorizableTupleScheme();
+    }
+  }
+
+  private static class TSentryAuthorizableTupleScheme extends TupleScheme<TSentryAuthorizable> {
+
+    @Override
+    public void write(org.apache.thrift.protocol.TProtocol prot, TSentryAuthorizable struct) throws org.apache.thrift.TException {
+      TTupleProtocol oprot = (TTupleProtocol) prot;
+      oprot.writeString(struct.server);
+      BitSet optionals = new BitSet();
+      if (struct.isSetUri()) {
+        optionals.set(0);
+      }
+      if (struct.isSetDb()) {
+        optionals.set(1);
+      }
+      if (struct.isSetTable()) {
+        optionals.set(2);
+      }
+      oprot.writeBitSet(optionals, 3);
+      if (struct.isSetUri()) {
+        oprot.writeString(struct.uri);
+      }
+      if (struct.isSetDb()) {
+        oprot.writeString(struct.db);
+      }
+      if (struct.isSetTable()) {
+        oprot.writeString(struct.table);
+      }
+    }
+
+    @Override
+    public void read(org.apache.thrift.protocol.TProtocol prot, TSentryAuthorizable struct) throws org.apache.thrift.TException {
+      TTupleProtocol iprot = (TTupleProtocol) prot;
+      struct.server = iprot.readString();
+      struct.setServerIsSet(true);
+      BitSet incoming = iprot.readBitSet(3);
+      if (incoming.get(0)) {
+        struct.uri = iprot.readString();
+        struct.setUriIsSet(true);
+      }
+      if (incoming.get(1)) {
+        struct.db = iprot.readString();
+        struct.setDbIsSet(true);
+      }
+      if (incoming.get(2)) {
+        struct.table = iprot.readString();
+        struct.setTableIsSet(true);
+      }
+    }
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
index 54c1d6d..b66037a 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SimpleDBProviderBackend.java
@@ -22,6 +22,7 @@ import java.util.Set;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.sentry.SentryUserException;
 import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.core.common.Authorizable;
 import org.apache.sentry.core.common.SentryConfigurationException;
 import org.apache.sentry.provider.common.ProviderBackend;
 import org.apache.sentry.provider.common.ProviderBackendContext;
@@ -31,6 +32,7 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
 
 public class SimpleDBProviderBackend implements ProviderBackend {
 
@@ -71,12 +73,12 @@ public class SimpleDBProviderBackend implements ProviderBackend {
    * {@inheritDoc}
    */
   @Override
-  public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet) {
+  public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet, Authorizable... authorizableHierarchy) {
     if (!initialized) {
       throw new IllegalStateException("Backend has not been properly initialized");
     }
     try {
-      return ImmutableSet.copyOf(policyServiceClient.listPrivilegesForProvider(groups, roleSet));
+      return ImmutableSet.copyOf(policyServiceClient.listPrivilegesForProvider(groups, roleSet, authorizableHierarchy));
     } catch (SentryUserException e) {
       String msg = "Unable to obtain privileges from server: " + e.getMessage();
       LOGGER.error(msg, e);
@@ -109,4 +111,4 @@ public class SimpleDBProviderBackend implements ProviderBackend {
     }
     // db provider does not implement validation
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a7b45622/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java
index 952ee78..5642f8b 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/model/MSentryPrivilege.java
@@ -41,7 +41,7 @@ public class MSentryPrivilege {
   private String URI;
   private String action;
   // roles this privilege is a part of
-  private final Set<MSentryRole> roles;
+  private Set<MSentryRole> roles;
   private long createTime;
   private String grantorPrincipal;