You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by sh...@apache.org on 2014/03/14 20:08:56 UTC

[6/6] git commit: SENTRY-142: Create database backed ProviderBackend (Brock Noland via Shreepadma Venugopalan)

SENTRY-142: Create database backed ProviderBackend (Brock Noland via Shreepadma Venugopalan)


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

Branch: refs/heads/master
Commit: 90cdbefd57b46281a593894b3a9fbf4686669228
Parents: 644e8be
Author: Shreepadma Venugopalan <sh...@apache.org>
Authored: Fri Mar 14 12:08:30 2014 -0700
Committer: Shreepadma Venugopalan <sh...@apache.org>
Committed: Fri Mar 14 12:08:30 2014 -0700

----------------------------------------------------------------------
 .../hive/HiveAuthzBindingPreExecHook.java       |   2 +-
 .../binding/hive/authz/HiveAuthzBinding.java    |  21 +-
 .../sentry/binding/hive/conf/HiveAuthzConf.java |   2 +-
 .../sentry/binding/solr/conf/SolrAuthzConf.java |   2 +-
 .../sentry/core/common/ActiveRoleSet.java       |   8 +
 .../sentry/policy/common/PolicyEngine.java      |   2 +
 .../sentry/policy/db/SimpleDBPolicyEngine.java  |   7 +
 ...sourceAuthorizationProviderGeneralCases.java |   2 +-
 .../policy/search/SimpleSearchPolicyEngine.java |   7 +
 ...SearchAuthorizationProviderGeneralCases.java |   2 +-
 sentry-provider/sentry-provider-common/pom.xml  |   4 +
 .../provider/common/AuthorizationProvider.java  |   5 +
 .../common/HadoopGroupMappingService.java       |  48 ++
 ...adoopGroupResourceAuthorizationProvider.java |  46 ++
 .../common/NoAuthorizationProvider.java         |   4 +
 .../sentry/provider/common/ProviderBackend.java |   2 +
 .../provider/common/ProviderConstants.java      |  38 +
 .../common/ResourceAuthorizationProvider.java   | 184 +++++
 ...adoopGroupResourceAuthorizationProvider.java |  44 +
 .../provider/common/TestGetGroupMapping.java    |  69 ++
 sentry-provider/sentry-provider-db/pom.xml      |   4 +
 .../db/service/thrift/SentryPolicyService.java  | 806 +++++++++++++++++++
 .../TAlterSentryRoleAddGroupsRequest.java       | 302 +++----
 .../TAlterSentryRoleDeleteGroupsRequest.java    | 414 ++++++++--
 .../TAlterSentryRoleGrantPrivilegeRequest.java  | 246 +++---
 .../TAlterSentryRoleRevokePrivilegeRequest.java | 246 +++---
 .../thrift/TCreateSentryRoleRequest.java        | 297 ++++---
 .../service/thrift/TDropSentryRoleRequest.java  | 268 +++---
 ...TListSentryPrivilegesForProviderRequest.java | 644 +++++++++++++++
 ...ListSentryPrivilegesForProviderResponse.java | 543 +++++++++++++
 .../service/thrift/TListSentryRolesRequest.java | 269 ++-----
 .../db/service/thrift/TSentryActiveRoleSet.java | 536 ++++++++++++
 .../provider/db/service/thrift/TSentryRole.java | 143 +---
 .../db/SentryAlreadyExistsException.java        |  27 +
 .../db/SentryInvalidInputException.java         |  27 +
 .../db/SentryNoSuchObjectException.java         |  27 +
 .../provider/db/SimpleDBProviderBackend.java    | 108 +++
 .../provider/db/service/model/MSentryGroup.java |  17 +-
 .../db/service/model/MSentryPrivilege.java      |  75 +-
 .../provider/db/service/model/MSentryRole.java  |  28 +-
 .../SentryAlreadyExistsException.java           |  27 -
 .../persistent/SentryInvalidInputException.java |  27 -
 .../persistent/SentryNoSuchObjectException.java |  27 -
 .../db/service/persistent/SentryStore.java      | 203 +++--
 .../thrift/SentryPolicyServiceClient.java       | 209 ++++-
 .../thrift/SentryPolicyStoreProcessor.java      |  39 +-
 .../sentry/service/thrift/ServiceConstants.java |  29 +
 .../apache/sentry/service/thrift/Status.java    |  34 +
 .../main/resources/sentry_policy_service.thrift | 109 ++-
 .../db/service/persistent/TestSentryStore.java  | 154 +++-
 .../TestSentryStoreToAuthorizable.java          |  86 ++
 .../thrift/TestSentryServiceIntegration.java    | 171 ++--
 .../thrift/SentryServiceIntegrationBase.java    |   9 +
 .../file/HadoopGroupMappingService.java         |  48 --
 ...adoopGroupResourceAuthorizationProvider.java |  45 --
 ...LocalGroupResourceAuthorizationProvider.java |   1 +
 .../provider/file/PolicyFileConstants.java      |  19 +-
 .../file/ResourceAuthorizationProvider.java     | 179 ----
 .../file/SimpleFileProviderBackend.java         |   5 +
 .../provider/file/TestGetGroupMapping.java      |  65 --
 60 files changed, 5106 insertions(+), 1906 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java
index bed7917..7859521 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingPreExecHook.java
@@ -49,7 +49,7 @@ public class HiveAuthzBindingPreExecHook implements ExecuteWithHookContext {
       // validate server level permissions permission for transforms
       if (qPlan.getQueryProperties().usesScript()) {
         if (hiveAuthzBinding == null) {
-          LOG.warn("No authorization binding fund, skipping the authorization for transform");
+          LOG.warn("No authorization binding found, skipping the authorization for transform");
           return;
         }
         List<List<DBModelAuthorizable>> inputHierarchy = new ArrayList<List<DBModelAuthorizable>> ();

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
index 65854c3..3be0d69 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/authz/HiveAuthzBinding.java
@@ -58,10 +58,12 @@ public class HiveAuthzBinding {
 
   private final Server authServer;
   private final AuthorizationProvider authProvider;
+  private volatile boolean open;
 
   public HiveAuthzBinding (HiveConf hiveConf, HiveAuthzConf authzConf) throws Exception {
     this.authServer = new Server(authzConf.get(AuthzConfVars.AUTHZ_SERVER_NAME.getVar()));
     this.authProvider = getAuthProvider(hiveConf, authzConf, authServer.getName());
+    this.open = true;
   }
 
   /**
@@ -83,6 +85,9 @@ public class HiveAuthzBinding {
    * @param conf
    */
   public void set (Configuration conf) {
+    if (!open) {
+      throw new IllegalStateException("Binding has been closed");
+    }
     String tagName = SessionState.get().getSessionId() + "_" + queryID.incrementAndGet();
     authzBindingMap.put(tagName, this);
     conf.set(HIVE_BINDING_TAG, tagName);
@@ -93,10 +98,15 @@ public class HiveAuthzBinding {
    * @param conf
    */
   public void clear(Configuration conf) {
+    if (!open) {
+      throw new IllegalStateException("Binding has been closed");
+    }
     String tagName = conf.get(HIVE_BINDING_TAG);
-    if (tagName == null) {
+    if (tagName != null) {
       authzBindingMap.remove(tagName);
     }
+    open = false;
+    authProvider.close();
   }
 
   // Instantiate the configured authz provider
@@ -170,6 +180,9 @@ public class HiveAuthzBinding {
   public void authorize(HiveOperation hiveOp, HiveAuthzPrivileges stmtAuthPrivileges,
       Subject subject, List<List<DBModelAuthorizable>> inputHierarchyList, List<List<DBModelAuthorizable>> outputHierarchyList )
           throws AuthorizationException {
+    if (!open) {
+      throw new IllegalStateException("Binding has been closed");
+    }
     boolean isDebug = LOG.isDebugEnabled();
     if(isDebug) {
       LOG.debug("Going to authorize statement " + hiveOp.name() +
@@ -223,6 +236,9 @@ public class HiveAuthzBinding {
   }
 
   public Server getAuthServer() {
+    if (!open) {
+      throw new IllegalStateException("Binding has been closed");
+    }
     return authServer;
   }
 
@@ -231,6 +247,9 @@ public class HiveAuthzBinding {
   }
 
   public List<String> getLastQueryPrivilegeErrors() {
+    if (!open) {
+      throw new IllegalStateException("Binding has been closed");
+    }
     return authProvider.getLastFailedPrivileges();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
index c4f12b5..336b925 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/conf/HiveAuthzConf.java
@@ -48,7 +48,7 @@ public class HiveAuthzConf extends Configuration {
    */
   public static enum AuthzConfVars {
     AUTHZ_PROVIDER("sentry.provider",
-      "org.apache.sentry.provider.file.ResourceAuthorizationProvider"),
+      "org.apache.sentry.provider.common.HadoopGroupResourceAuthorizationProvider"),
     AUTHZ_PROVIDER_RESOURCE("sentry.hive.provider.resource", ""),
     AUTHZ_PROVIDER_BACKEND("sentry.hive.provider.backend", "org.apache.sentry.provider.file.SimpleFileProviderBackend"),
     AUTHZ_POLICY_ENGINE("sentry.hive.policy.engine", "org.apache.sentry.policy.db.SimpleDBPolicyEngine"),

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/conf/SolrAuthzConf.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/conf/SolrAuthzConf.java b/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/conf/SolrAuthzConf.java
index 70983c4..2d7bae8 100644
--- a/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/conf/SolrAuthzConf.java
+++ b/sentry-binding/sentry-binding-solr/src/main/java/org/apache/sentry/binding/solr/conf/SolrAuthzConf.java
@@ -30,7 +30,7 @@ public class SolrAuthzConf extends Configuration {
    */
   public static enum AuthzConfVars {
     AUTHZ_PROVIDER("sentry.provider",
-      "org.apache.sentry.provider.file.ResourceAuthorizationProvider"),
+      "org.apache.sentry.provider.common.HadoopGroupResourceAuthorizationProvider"),
     AUTHZ_PROVIDER_RESOURCE("sentry.solr.provider.resource", ""),
     AUTHZ_PROVIDER_BACKEND("sentry.solr.provider.backend", "org.apache.sentry.provider.file.SimpleFileProviderBackend"),
     AUTHZ_POLICY_ENGINE("sentry.solr.policy.engine", "org.apache.sentry.policy.search.SimpleSearchPolicyEngine");

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/ActiveRoleSet.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/ActiveRoleSet.java b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/ActiveRoleSet.java
index c1f1f66..a0199fa 100644
--- a/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/ActiveRoleSet.java
+++ b/sentry-core/sentry-core-common/src/main/java/org/apache/sentry/core/common/ActiveRoleSet.java
@@ -58,6 +58,14 @@ public class ActiveRoleSet {
     return allRoles || roles.contains(role.toLowerCase());
   }
 
+  public boolean isAll() {
+    return allRoles;
+  }
+
+  public Set<String> getRoles() {
+    return roles;
+  }
+
   @Override
   public String toString() {
     StringBuilder builder = new StringBuilder("ActiveRoleSet = [ roles = ");

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/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 512e28e..c378a38 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
@@ -50,5 +50,7 @@ public interface PolicyEngine {
   public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet)
       throws SentryConfigurationException;
 
+  public void close();
+
   public void validatePolicy(boolean strictValidation) throws SentryConfigurationException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/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 e67daf4..a95ef7b 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
@@ -77,6 +77,13 @@ public class SimpleDBPolicyEngine implements PolicyEngine {
     this.providerBackend.validatePolicy(strictValidation);
   }
 
+  @Override
+  public void close() {
+    if (providerBackend != null) {
+      providerBackend.close();
+    }
+  }
+
   public static ImmutableList<PrivilegeValidator> createPrivilegeValidators(String serverName) {
     return ImmutableList.<PrivilegeValidator>of(new ServersAllIsInvalid(), new DatabaseMustMatch(),
         new DatabaseRequiredInPrivilege(), new ServerNameMustMatch(serverName));

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderGeneralCases.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderGeneralCases.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderGeneralCases.java
index 469be14..53b83a5 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderGeneralCases.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/TestResourceAuthorizationProviderGeneralCases.java
@@ -36,9 +36,9 @@ import org.apache.sentry.core.model.db.Database;
 import org.apache.sentry.core.model.db.Server;
 import org.apache.sentry.core.model.db.Table;
 import org.apache.sentry.provider.common.MockGroupMappingServiceProvider;
+import org.apache.sentry.provider.common.ResourceAuthorizationProvider;
 import org.apache.sentry.provider.file.HadoopGroupResourceAuthorizationProvider;
 import org.apache.sentry.provider.file.PolicyFiles;
-import org.apache.sentry.provider.file.ResourceAuthorizationProvider;
 import org.junit.After;
 import org.junit.Test;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/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 728e356..8adcb6f 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
@@ -81,4 +81,11 @@ public class SimpleSearchPolicyEngine implements PolicyEngine {
   public static ImmutableList<PrivilegeValidator> createPrivilegeValidators() {
     return ImmutableList.<PrivilegeValidator>of(new CollectionRequiredInPrivilege());
   }
+
+  @Override
+  public void close() {
+    if (providerBackend != null) {
+      providerBackend.close();
+    }
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderGeneralCases.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderGeneralCases.java b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderGeneralCases.java
index 6f36243..bdb1c96 100644
--- a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderGeneralCases.java
+++ b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/TestSearchAuthorizationProviderGeneralCases.java
@@ -33,9 +33,9 @@ import org.apache.sentry.core.common.Subject;
 import org.apache.sentry.core.model.search.Collection;
 import org.apache.sentry.core.model.search.SearchModelAction;
 import org.apache.sentry.provider.common.MockGroupMappingServiceProvider;
+import org.apache.sentry.provider.common.ResourceAuthorizationProvider;
 import org.apache.sentry.provider.file.HadoopGroupResourceAuthorizationProvider;
 import org.apache.sentry.provider.file.PolicyFiles;
-import org.apache.sentry.provider.file.ResourceAuthorizationProvider;
 import org.junit.After;
 import org.junit.Test;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-provider/sentry-provider-common/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/pom.xml b/sentry-provider/sentry-provider-common/pom.xml
index 1e9dc1b..7c17950 100644
--- a/sentry-provider/sentry-provider-common/pom.xml
+++ b/sentry-provider/sentry-provider-common/pom.xml
@@ -34,6 +34,10 @@ limitations under the License.
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+    </dependency>
+    <dependency>
       <groupId>org.apache.sentry</groupId>
       <artifactId>sentry-core-common</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java
index cd6f8a1..de774f4 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/AuthorizationProvider.java
@@ -83,4 +83,9 @@ public interface AuthorizationProvider {
    * @return
    */
   public List<String> getLastFailedPrivileges();
+
+  /**
+   * Frees any resources held by the the provider
+   */
+  public void close();
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/HadoopGroupMappingService.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/HadoopGroupMappingService.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/HadoopGroupMappingService.java
new file mode 100644
index 0000000..6c8fa95
--- /dev/null
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/HadoopGroupMappingService.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sentry.provider.common;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.security.Groups;
+import org.apache.sentry.provider.common.GroupMappingService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HadoopGroupMappingService implements GroupMappingService {
+
+  private static final Logger LOGGER = LoggerFactory
+      .getLogger(HadoopGroupMappingService.class);
+  private final Groups groups;
+
+  public HadoopGroupMappingService(Groups groups) {
+    this.groups = groups;
+  }
+
+  @Override
+  public Set<String> getGroups(String user) {
+    try {
+      return new HashSet<String>(groups.getGroups(user));
+    } catch (IOException e) {
+      LOGGER.warn("Unable to obtain groups for " + user, e);
+    }
+    return Collections.emptySet();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/HadoopGroupResourceAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/HadoopGroupResourceAuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/HadoopGroupResourceAuthorizationProvider.java
new file mode 100644
index 0000000..47ba77a
--- /dev/null
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/HadoopGroupResourceAuthorizationProvider.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.sentry.provider.common;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.Groups;
+import org.apache.sentry.policy.common.PolicyEngine;
+import org.apache.sentry.provider.common.GroupMappingService;
+import org.apache.sentry.provider.common.HadoopGroupMappingService;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class HadoopGroupResourceAuthorizationProvider extends
+  ResourceAuthorizationProvider {
+
+  // resource parameter present so that other AuthorizationProviders (e.g.
+  // LocalGroupResourceAuthorizationProvider) has the same constructor params.
+  public HadoopGroupResourceAuthorizationProvider(String resource, PolicyEngine policy) throws IOException {
+    this(policy, new HadoopGroupMappingService(
+        Groups.getUserToGroupsMappingService(new Configuration())));
+  }
+
+  @VisibleForTesting
+  public HadoopGroupResourceAuthorizationProvider(PolicyEngine policy,
+      GroupMappingService groupService) {
+    super(policy, groupService);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java
index ed32224..a814527 100644
--- a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/NoAuthorizationProvider.java
@@ -63,4 +63,8 @@ public class NoAuthorizationProvider implements AuthorizationProvider {
     return new ArrayList<String>();
   }
 
+  @Override
+  public void close() {
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/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 6d6da25..26c4878 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
@@ -56,4 +56,6 @@ public interface ProviderBackend {
    * @throws SentryConfigurationException
    */
   public void validatePolicy(boolean strictValidation) throws SentryConfigurationException;
+
+  public void close();
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderConstants.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderConstants.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderConstants.java
new file mode 100644
index 0000000..c6f7e2c
--- /dev/null
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ProviderConstants.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sentry.provider.common;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Splitter;
+
+public class ProviderConstants {
+
+  public static final String ROLE_SEPARATOR = ",";
+  public static final String AUTHORIZABLE_SEPARATOR = "->";
+  public static final String KV_SEPARATOR = "=";
+
+  public static final Splitter ROLE_SPLITTER = Splitter.on(ROLE_SEPARATOR);
+  public static final Splitter AUTHORIZABLE_SPLITTER = Splitter.on(AUTHORIZABLE_SEPARATOR);
+  public static final Splitter KV_SPLITTER = Splitter.on(KV_SEPARATOR);
+  public static final Joiner ROLE_JOINER = Joiner.on(ROLE_SEPARATOR);
+  public static final Joiner AUTHORIZABLE_JOINER = Joiner.on(AUTHORIZABLE_SEPARATOR);
+  public static final Joiner KV_JOINER = Joiner.on(KV_SEPARATOR);
+
+  // TODO change to privilege
+  public static final String PRIVILEGE_NAME = "action";
+  public static final String PRIVILEGE_PREFIX = (PRIVILEGE_NAME + KV_SEPARATOR).toLowerCase();
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/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
new file mode 100644
index 0000000..e1e7f4a
--- /dev/null
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/common/ResourceAuthorizationProvider.java
@@ -0,0 +1,184 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sentry.provider.common;
+
+import static org.apache.sentry.provider.common.ProviderConstants.AUTHORIZABLE_JOINER;
+import static org.apache.sentry.provider.common.ProviderConstants.KV_JOINER;
+import static org.apache.sentry.provider.common.ProviderConstants.PRIVILEGE_NAME;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.sentry.core.common.Action;
+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.core.common.Subject;
+import org.apache.sentry.policy.common.Privilege;
+import org.apache.sentry.policy.common.PrivilegeFactory;
+import org.apache.sentry.policy.common.PolicyEngine;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+
+public abstract class ResourceAuthorizationProvider implements AuthorizationProvider {
+  private static final Logger LOGGER = LoggerFactory
+      .getLogger(ResourceAuthorizationProvider.class);
+  private final GroupMappingService groupService;
+  private final PolicyEngine policy;
+  private final PrivilegeFactory privilegeFactory;
+  private final ThreadLocal<List<String>> lastFailedPrivileges;
+
+  public ResourceAuthorizationProvider(PolicyEngine policy,
+      GroupMappingService groupService) {
+    this.policy = policy;
+    this.groupService = groupService;
+    this.privilegeFactory = policy.getPrivilegeFactory();
+    this.lastFailedPrivileges = new ThreadLocal<List<String>>() {
+      @Override
+      protected List<String> initialValue() {
+        return new ArrayList<String>();
+      }
+    };
+  }
+
+  /***
+   * @param subject: UserID to validate privileges
+   * @param authorizableHierarchy : List of object according to namespace hierarchy.
+   *        eg. Server->Db->Table or Server->Function
+   *        The privileges will be validated from the higher to lower scope
+   * @param actions : Privileges to validate
+   * @return
+   *        True if the subject is authorized to perform requested action on the given object
+   */
+  @Override
+  public boolean hasAccess(Subject subject, List<? extends Authorizable> authorizableHierarchy,
+      Set<? extends Action> actions, ActiveRoleSet roleSet) {
+    if(LOGGER.isDebugEnabled()) {
+      LOGGER.debug("Authorization Request for " + subject + " " +
+          authorizableHierarchy + " and " + actions);
+    }
+    Preconditions.checkNotNull(subject, "Subject cannot be null");
+    Preconditions.checkNotNull(authorizableHierarchy, "Authorizable cannot be null");
+    Preconditions.checkArgument(!authorizableHierarchy.isEmpty(), "Authorizable cannot be empty");
+    Preconditions.checkNotNull(actions, "Actions cannot be null");
+    Preconditions.checkArgument(!actions.isEmpty(), "Actions cannot be empty");
+    Preconditions.checkNotNull(roleSet, "ActiveRoleSet cannot be null");
+    return doHasAccess(subject, authorizableHierarchy, actions, roleSet);
+  }
+
+  private boolean doHasAccess(Subject subject,
+      List<? extends Authorizable> authorizables, Set<? extends Action> actions,
+      ActiveRoleSet roleSet) {
+    Set<String> groups =  getGroups(subject);
+    Set<String> hierarchy = new HashSet<String>();
+    for (Authorizable authorizable : authorizables) {
+      hierarchy.add(KV_JOINER.join(authorizable.getTypeName(), authorizable.getName()));
+    }
+    Iterable<Privilege> privileges = getPrivileges(groups, roleSet);
+    List<String> requestPrivileges = buildPermissions(authorizables, actions);
+    lastFailedPrivileges.get().clear();
+
+    for (String requestPrivilege : requestPrivileges) {
+      for (Privilege permission : privileges) {
+        /*
+         * Does the permission granted in the policy file imply the requested action?
+         */
+        boolean result = permission.implies(privilegeFactory.createPrivilege(requestPrivilege));
+        if(LOGGER.isDebugEnabled()) {
+          LOGGER.debug("ProviderPrivilege {}, RequestPrivilege {}, RoleSet, {}, Result {}",
+              new Object[]{ permission, requestPrivilege, roleSet, result});
+        }
+        if (result) {
+          return true;
+        }
+      }
+    }
+    lastFailedPrivileges.get().addAll(requestPrivileges);
+    return false;
+  }
+
+  private Iterable<Privilege> getPrivileges(Set<String> groups, ActiveRoleSet roleSet) {
+    return Iterables.transform(policy.getPrivileges(groups, roleSet),
+        new Function<String, Privilege>() {
+      @Override
+      public Privilege apply(String privilege) {
+        return privilegeFactory.createPrivilege(privilege);
+      }
+    });
+  }
+
+  @Override
+  public GroupMappingService getGroupMapping() {
+    return groupService;
+  }
+
+  private Set<String> getGroups(Subject subject) {
+    return groupService.getGroups(subject.getName());
+  }
+
+  @Override
+  public void validateResource(boolean strictValidation) throws SentryConfigurationException {
+    policy.validatePolicy(strictValidation);
+  }
+
+  @Override
+  public Set<String> listPrivilegesForSubject(Subject subject) throws SentryConfigurationException {
+    return policy.getPrivileges(getGroups(subject), ActiveRoleSet.ALL);
+  }
+
+  @Override
+  public Set<String> listPrivilegesForGroup(String groupName) throws SentryConfigurationException {
+    return policy.getPrivileges(Sets.newHashSet(groupName), ActiveRoleSet.ALL);
+  }
+
+  @Override
+  public List<String> getLastFailedPrivileges() {
+    return lastFailedPrivileges.get();
+  }
+
+  @Override
+  public void close() {
+    if (policy != null) {
+      policy.close();
+    }
+  }
+
+  private List<String> buildPermissions(List<? extends Authorizable> authorizables,
+      Set<? extends Action> actions) {
+    List<String> hierarchy = new ArrayList<String>();
+    List<String> requestedPermissions = new ArrayList<String>();
+
+    for (Authorizable authorizable : authorizables) {
+      hierarchy.add(KV_JOINER.join(authorizable.getTypeName(), authorizable.getName()));
+    }
+
+    for (Action action : actions) {
+      String requestPermission = AUTHORIZABLE_JOINER.join(hierarchy);
+      requestPermission = AUTHORIZABLE_JOINER.join(requestPermission,
+          KV_JOINER.join(PRIVILEGE_NAME, action.getValue()));
+      requestedPermissions.add(requestPermission);
+    }
+    return requestedPermissions;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/file/HadoopGroupResourceAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/file/HadoopGroupResourceAuthorizationProvider.java b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/file/HadoopGroupResourceAuthorizationProvider.java
new file mode 100644
index 0000000..1cbc70c
--- /dev/null
+++ b/sentry-provider/sentry-provider-common/src/main/java/org/apache/sentry/provider/file/HadoopGroupResourceAuthorizationProvider.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.sentry.provider.file;
+
+import java.io.IOException;
+
+import org.apache.sentry.policy.common.PolicyEngine;
+import org.apache.sentry.provider.common.GroupMappingService;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Kept for backwards compatibility
+ */
+@Deprecated
+public class HadoopGroupResourceAuthorizationProvider extends
+  org.apache.sentry.provider.common.HadoopGroupResourceAuthorizationProvider {
+
+  public HadoopGroupResourceAuthorizationProvider(String resource, PolicyEngine policy) throws IOException {
+    super(resource, policy);
+  }
+
+  @VisibleForTesting
+  public HadoopGroupResourceAuthorizationProvider(PolicyEngine policy,
+      GroupMappingService groupService) {
+    super(policy, groupService);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/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
new file mode 100644
index 0000000..ece740b
--- /dev/null
+++ b/sentry-provider/sentry-provider-common/src/test/java/org/apache/sentry/provider/common/TestGetGroupMapping.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sentry.provider.common;
+
+import static org.junit.Assert.assertSame;
+
+import java.util.Set;
+
+import org.apache.sentry.core.common.SentryConfigurationException;
+import org.apache.sentry.core.common.ActiveRoleSet;
+import org.apache.sentry.policy.common.PrivilegeFactory;
+import org.apache.sentry.policy.common.PolicyEngine;
+import org.apache.sentry.provider.common.GroupMappingService;
+import org.apache.sentry.provider.common.ResourceAuthorizationProvider;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
+public class TestGetGroupMapping {
+
+  private static class TestResourceAuthorizationProvider extends ResourceAuthorizationProvider {
+    public TestResourceAuthorizationProvider(PolicyEngine policy,
+      GroupMappingService groupService) {
+      super(policy, groupService);
+    }
+  };
+
+  @Test
+  public void testResourceAuthorizationProvider() {
+    final Set<String> set = Sets.newHashSet("a", "b", "c");
+    GroupMappingService mappingService = new GroupMappingService() {
+      public Set<String> getGroups(String user) { return set; }
+    };
+    PolicyEngine policyEngine = new PolicyEngine() {
+      public PrivilegeFactory getPrivilegeFactory() { return null; }
+
+      public ImmutableSet<String> getPrivileges(Set<String> groups, ActiveRoleSet roleSet) {
+        return ImmutableSet.of();
+      }
+
+      public void validatePolicy(boolean strictValidation)
+          throws SentryConfigurationException {
+        return;
+      }
+
+      @Override
+      public void close() {}
+    };
+
+    TestResourceAuthorizationProvider authProvider =
+      new TestResourceAuthorizationProvider(policyEngine, mappingService);
+    assertSame(authProvider.getGroupMapping(), mappingService);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-provider/sentry-provider-db/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/pom.xml b/sentry-provider/sentry-provider-db/pom.xml
index aa511c8..d82a564 100644
--- a/sentry-provider/sentry-provider-db/pom.xml
+++ b/sentry-provider/sentry-provider-db/pom.xml
@@ -71,6 +71,10 @@ limitations under the License.
     </dependency>
     <dependency>
       <groupId>org.apache.sentry</groupId>
+      <artifactId>sentry-core-model-db</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.sentry</groupId>
       <artifactId>sentry-provider-common</artifactId>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/90cdbefd/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java
index 6f02595..cc6eb4f 100644
--- a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/provider/db/service/thrift/SentryPolicyService.java
@@ -51,6 +51,8 @@ public class SentryPolicyService {
 
     public TListSentryRolesResponse list_sentry_roles_by_role_name(TListSentryRolesRequest request) throws org.apache.thrift.TException;
 
+    public TListSentryPrivilegesForProviderResponse list_sentry_privileges_for_provider(TListSentryPrivilegesForProviderRequest request) throws org.apache.thrift.TException;
+
   }
 
   public interface AsyncIface {
@@ -71,6 +73,8 @@ public class SentryPolicyService {
 
     public void list_sentry_roles_by_role_name(TListSentryRolesRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.list_sentry_roles_by_role_name_call> resultHandler) throws org.apache.thrift.TException;
 
+    public void list_sentry_privileges_for_provider(TListSentryPrivilegesForProviderRequest request, org.apache.thrift.async.AsyncMethodCallback<AsyncClient.list_sentry_privileges_for_provider_call> resultHandler) throws org.apache.thrift.TException;
+
   }
 
   public static class Client extends org.apache.thrift.TServiceClient implements Iface {
@@ -277,6 +281,29 @@ public class SentryPolicyService {
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "list_sentry_roles_by_role_name failed: unknown result");
     }
 
+    public TListSentryPrivilegesForProviderResponse list_sentry_privileges_for_provider(TListSentryPrivilegesForProviderRequest request) throws org.apache.thrift.TException
+    {
+      send_list_sentry_privileges_for_provider(request);
+      return recv_list_sentry_privileges_for_provider();
+    }
+
+    public void send_list_sentry_privileges_for_provider(TListSentryPrivilegesForProviderRequest request) throws org.apache.thrift.TException
+    {
+      list_sentry_privileges_for_provider_args args = new list_sentry_privileges_for_provider_args();
+      args.setRequest(request);
+      sendBase("list_sentry_privileges_for_provider", args);
+    }
+
+    public TListSentryPrivilegesForProviderResponse recv_list_sentry_privileges_for_provider() throws org.apache.thrift.TException
+    {
+      list_sentry_privileges_for_provider_result result = new list_sentry_privileges_for_provider_result();
+      receiveBase(result, "list_sentry_privileges_for_provider");
+      if (result.isSetSuccess()) {
+        return result.success;
+      }
+      throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "list_sentry_privileges_for_provider failed: unknown result");
+    }
+
   }
   public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface {
     public static class Factory implements org.apache.thrift.async.TAsyncClientFactory<AsyncClient> {
@@ -551,6 +578,38 @@ public class SentryPolicyService {
       }
     }
 
+    public void list_sentry_privileges_for_provider(TListSentryPrivilegesForProviderRequest request, org.apache.thrift.async.AsyncMethodCallback<list_sentry_privileges_for_provider_call> resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      list_sentry_privileges_for_provider_call method_call = new list_sentry_privileges_for_provider_call(request, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class list_sentry_privileges_for_provider_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private TListSentryPrivilegesForProviderRequest request;
+      public list_sentry_privileges_for_provider_call(TListSentryPrivilegesForProviderRequest request, org.apache.thrift.async.AsyncMethodCallback<list_sentry_privileges_for_provider_call> resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, false);
+        this.request = request;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("list_sentry_privileges_for_provider", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        list_sentry_privileges_for_provider_args args = new list_sentry_privileges_for_provider_args();
+        args.setRequest(request);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public TListSentryPrivilegesForProviderResponse getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+        return (new Client(prot)).recv_list_sentry_privileges_for_provider();
+      }
+    }
+
   }
 
   public static class Processor<I extends Iface> extends org.apache.thrift.TBaseProcessor<I> implements org.apache.thrift.TProcessor {
@@ -572,6 +631,7 @@ public class SentryPolicyService {
       processMap.put("alter_sentry_role_delete_groups", new alter_sentry_role_delete_groups());
       processMap.put("list_sentry_roles_by_group", new list_sentry_roles_by_group());
       processMap.put("list_sentry_roles_by_role_name", new list_sentry_roles_by_role_name());
+      processMap.put("list_sentry_privileges_for_provider", new list_sentry_privileges_for_provider());
       return processMap;
     }
 
@@ -735,6 +795,26 @@ public class SentryPolicyService {
       }
     }
 
+    public static class list_sentry_privileges_for_provider<I extends Iface> extends org.apache.thrift.ProcessFunction<I, list_sentry_privileges_for_provider_args> {
+      public list_sentry_privileges_for_provider() {
+        super("list_sentry_privileges_for_provider");
+      }
+
+      public list_sentry_privileges_for_provider_args getEmptyArgsInstance() {
+        return new list_sentry_privileges_for_provider_args();
+      }
+
+      protected boolean isOneway() {
+        return false;
+      }
+
+      public list_sentry_privileges_for_provider_result getResult(I iface, list_sentry_privileges_for_provider_args args) throws org.apache.thrift.TException {
+        list_sentry_privileges_for_provider_result result = new list_sentry_privileges_for_provider_result();
+        result.success = iface.list_sentry_privileges_for_provider(args.request);
+        return result;
+      }
+    }
+
   }
 
   public static class create_sentry_role_args implements org.apache.thrift.TBase<create_sentry_role_args, create_sentry_role_args._Fields>, java.io.Serializable, Cloneable   {
@@ -6545,4 +6625,730 @@ public class SentryPolicyService {
 
   }
 
+  public static class list_sentry_privileges_for_provider_args implements org.apache.thrift.TBase<list_sentry_privileges_for_provider_args, list_sentry_privileges_for_provider_args._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("list_sentry_privileges_for_provider_args");
+
+    private static final org.apache.thrift.protocol.TField REQUEST_FIELD_DESC = new org.apache.thrift.protocol.TField("request", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new list_sentry_privileges_for_provider_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new list_sentry_privileges_for_provider_argsTupleSchemeFactory());
+    }
+
+    private TListSentryPrivilegesForProviderRequest request; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      REQUEST((short)1, "request");
+
+      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: // REQUEST
+            return REQUEST;
+          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
+    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.REQUEST, new org.apache.thrift.meta_data.FieldMetaData("request", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TListSentryPrivilegesForProviderRequest.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(list_sentry_privileges_for_provider_args.class, metaDataMap);
+    }
+
+    public list_sentry_privileges_for_provider_args() {
+    }
+
+    public list_sentry_privileges_for_provider_args(
+      TListSentryPrivilegesForProviderRequest request)
+    {
+      this();
+      this.request = request;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public list_sentry_privileges_for_provider_args(list_sentry_privileges_for_provider_args other) {
+      if (other.isSetRequest()) {
+        this.request = new TListSentryPrivilegesForProviderRequest(other.request);
+      }
+    }
+
+    public list_sentry_privileges_for_provider_args deepCopy() {
+      return new list_sentry_privileges_for_provider_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.request = null;
+    }
+
+    public TListSentryPrivilegesForProviderRequest getRequest() {
+      return this.request;
+    }
+
+    public void setRequest(TListSentryPrivilegesForProviderRequest request) {
+      this.request = request;
+    }
+
+    public void unsetRequest() {
+      this.request = null;
+    }
+
+    /** Returns true if field request is set (has been assigned a value) and false otherwise */
+    public boolean isSetRequest() {
+      return this.request != null;
+    }
+
+    public void setRequestIsSet(boolean value) {
+      if (!value) {
+        this.request = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case REQUEST:
+        if (value == null) {
+          unsetRequest();
+        } else {
+          setRequest((TListSentryPrivilegesForProviderRequest)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case REQUEST:
+        return getRequest();
+
+      }
+      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 REQUEST:
+        return isSetRequest();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof list_sentry_privileges_for_provider_args)
+        return this.equals((list_sentry_privileges_for_provider_args)that);
+      return false;
+    }
+
+    public boolean equals(list_sentry_privileges_for_provider_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_request = true && this.isSetRequest();
+      boolean that_present_request = true && that.isSetRequest();
+      if (this_present_request || that_present_request) {
+        if (!(this_present_request && that_present_request))
+          return false;
+        if (!this.request.equals(that.request))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_request = true && (isSetRequest());
+      builder.append(present_request);
+      if (present_request)
+        builder.append(request);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(list_sentry_privileges_for_provider_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      list_sentry_privileges_for_provider_args typedOther = (list_sentry_privileges_for_provider_args)other;
+
+      lastComparison = Boolean.valueOf(isSetRequest()).compareTo(typedOther.isSetRequest());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetRequest()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.request, typedOther.request);
+        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("list_sentry_privileges_for_provider_args(");
+      boolean first = true;
+
+      sb.append("request:");
+      if (this.request == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.request);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (request != null) {
+        request.validate();
+      }
+    }
+
+    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 list_sentry_privileges_for_provider_argsStandardSchemeFactory implements SchemeFactory {
+      public list_sentry_privileges_for_provider_argsStandardScheme getScheme() {
+        return new list_sentry_privileges_for_provider_argsStandardScheme();
+      }
+    }
+
+    private static class list_sentry_privileges_for_provider_argsStandardScheme extends StandardScheme<list_sentry_privileges_for_provider_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, list_sentry_privileges_for_provider_args 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: // REQUEST
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.request = new TListSentryPrivilegesForProviderRequest();
+                struct.request.read(iprot);
+                struct.setRequestIsSet(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, list_sentry_privileges_for_provider_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.request != null) {
+          oprot.writeFieldBegin(REQUEST_FIELD_DESC);
+          struct.request.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class list_sentry_privileges_for_provider_argsTupleSchemeFactory implements SchemeFactory {
+      public list_sentry_privileges_for_provider_argsTupleScheme getScheme() {
+        return new list_sentry_privileges_for_provider_argsTupleScheme();
+      }
+    }
+
+    private static class list_sentry_privileges_for_provider_argsTupleScheme extends TupleScheme<list_sentry_privileges_for_provider_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, list_sentry_privileges_for_provider_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetRequest()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetRequest()) {
+          struct.request.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, list_sentry_privileges_for_provider_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.request = new TListSentryPrivilegesForProviderRequest();
+          struct.request.read(iprot);
+          struct.setRequestIsSet(true);
+        }
+      }
+    }
+
+  }
+
+  public static class list_sentry_privileges_for_provider_result implements org.apache.thrift.TBase<list_sentry_privileges_for_provider_result, list_sentry_privileges_for_provider_result._Fields>, java.io.Serializable, Cloneable   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("list_sentry_privileges_for_provider_result");
+
+    private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new list_sentry_privileges_for_provider_resultStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new list_sentry_privileges_for_provider_resultTupleSchemeFactory());
+    }
+
+    private TListSentryPrivilegesForProviderResponse success; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      SUCCESS((short)0, "success");
+
+      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 0: // SUCCESS
+            return SUCCESS;
+          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
+    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.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TListSentryPrivilegesForProviderResponse.class)));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(list_sentry_privileges_for_provider_result.class, metaDataMap);
+    }
+
+    public list_sentry_privileges_for_provider_result() {
+    }
+
+    public list_sentry_privileges_for_provider_result(
+      TListSentryPrivilegesForProviderResponse success)
+    {
+      this();
+      this.success = success;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public list_sentry_privileges_for_provider_result(list_sentry_privileges_for_provider_result other) {
+      if (other.isSetSuccess()) {
+        this.success = new TListSentryPrivilegesForProviderResponse(other.success);
+      }
+    }
+
+    public list_sentry_privileges_for_provider_result deepCopy() {
+      return new list_sentry_privileges_for_provider_result(this);
+    }
+
+    @Override
+    public void clear() {
+      this.success = null;
+    }
+
+    public TListSentryPrivilegesForProviderResponse getSuccess() {
+      return this.success;
+    }
+
+    public void setSuccess(TListSentryPrivilegesForProviderResponse success) {
+      this.success = success;
+    }
+
+    public void unsetSuccess() {
+      this.success = null;
+    }
+
+    /** Returns true if field success is set (has been assigned a value) and false otherwise */
+    public boolean isSetSuccess() {
+      return this.success != null;
+    }
+
+    public void setSuccessIsSet(boolean value) {
+      if (!value) {
+        this.success = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case SUCCESS:
+        if (value == null) {
+          unsetSuccess();
+        } else {
+          setSuccess((TListSentryPrivilegesForProviderResponse)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case SUCCESS:
+        return getSuccess();
+
+      }
+      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 SUCCESS:
+        return isSetSuccess();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof list_sentry_privileges_for_provider_result)
+        return this.equals((list_sentry_privileges_for_provider_result)that);
+      return false;
+    }
+
+    public boolean equals(list_sentry_privileges_for_provider_result that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_success = true && this.isSetSuccess();
+      boolean that_present_success = true && that.isSetSuccess();
+      if (this_present_success || that_present_success) {
+        if (!(this_present_success && that_present_success))
+          return false;
+        if (!this.success.equals(that.success))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      HashCodeBuilder builder = new HashCodeBuilder();
+
+      boolean present_success = true && (isSetSuccess());
+      builder.append(present_success);
+      if (present_success)
+        builder.append(success);
+
+      return builder.toHashCode();
+    }
+
+    public int compareTo(list_sentry_privileges_for_provider_result other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+      list_sentry_privileges_for_provider_result typedOther = (list_sentry_privileges_for_provider_result)other;
+
+      lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(typedOther.isSetSuccess());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetSuccess()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, typedOther.success);
+        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("list_sentry_privileges_for_provider_result(");
+      boolean first = true;
+
+      sb.append("success:");
+      if (this.success == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.success);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (success != null) {
+        success.validate();
+      }
+    }
+
+    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 list_sentry_privileges_for_provider_resultStandardSchemeFactory implements SchemeFactory {
+      public list_sentry_privileges_for_provider_resultStandardScheme getScheme() {
+        return new list_sentry_privileges_for_provider_resultStandardScheme();
+      }
+    }
+
+    private static class list_sentry_privileges_for_provider_resultStandardScheme extends StandardScheme<list_sentry_privileges_for_provider_result> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, list_sentry_privileges_for_provider_result 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 0: // SUCCESS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.success = new TListSentryPrivilegesForProviderResponse();
+                struct.success.read(iprot);
+                struct.setSuccessIsSet(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, list_sentry_privileges_for_provider_result struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.success != null) {
+          oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
+          struct.success.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class list_sentry_privileges_for_provider_resultTupleSchemeFactory implements SchemeFactory {
+      public list_sentry_privileges_for_provider_resultTupleScheme getScheme() {
+        return new list_sentry_privileges_for_provider_resultTupleScheme();
+      }
+    }
+
+    private static class list_sentry_privileges_for_provider_resultTupleScheme extends TupleScheme<list_sentry_privileges_for_provider_result> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, list_sentry_privileges_for_provider_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetSuccess()) {
+          optionals.set(0);
+        }
+        oprot.writeBitSet(optionals, 1);
+        if (struct.isSetSuccess()) {
+          struct.success.write(oprot);
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, list_sentry_privileges_for_provider_result struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(1);
+        if (incoming.get(0)) {
+          struct.success = new TListSentryPrivilegesForProviderResponse();
+          struct.success.read(iprot);
+          struct.setSuccessIsSet(true);
+        }
+      }
+    }
+
+  }
+
 }