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/19 00:30:09 UTC

[2/2] git commit: SENTRY-129: Implement Hive Sentry Authz DDL Task Factory (Brock Noland via Shreepadma Venugopalan)

SENTRY-129: Implement Hive Sentry Authz DDL Task Factory (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/63c134f3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-sentry/tree/63c134f3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-sentry/diff/63c134f3

Branch: refs/heads/master
Commit: 63c134f360760a17434b33c26a612a846271523a
Parents: fbf042e
Author: Shreepadma Venugopalan <sh...@apache.org>
Authored: Tue Mar 18 16:29:40 2014 -0700
Committer: Shreepadma Venugopalan <sh...@apache.org>
Committed: Tue Mar 18 16:29:40 2014 -0700

----------------------------------------------------------------------
 pom.xml                                         |  10 +-
 sentry-binding/sentry-binding-hive/pom.xml      |  10 +
 .../apache/hadoop/hive/SentryHiveConstants.java |  34 ++
 .../hive/ql/exec/SentryGrantRevokeTask.java     | 496 +++++++++++++++++++
 .../binding/hive/HiveAuthzBindingHook.java      |  15 +-
 .../hive/HiveAuthzBindingSessionHook.java       |  41 +-
 .../SentryHiveAuthorizationTaskFactoryImpl.java | 330 ++++++++++++
 .../binding/hive/authz/HiveAuthzBinding.java    |   9 +-
 .../TestSentryHiveAuthorizationTaskFactory.java | 374 ++++++++++++++
 .../sentry/policy/db/DBPolicyFileBackend.java   |   3 +-
 .../policy/search/SearchPolicyFileBackend.java  |   3 +-
 .../thrift/sentry_common_serviceConstants.java  |   2 +
 .../db/SentryAccessDeniedException.java         |  27 +
 .../provider/db/SimpleDBProviderBackend.java    |   4 -
 .../thrift/SentryPolicyServiceClient.java       |  51 +-
 .../thrift/SentryPolicyStoreProcessor.java      |  49 ++
 .../sentry/service/thrift/GSSCallback.java      |   5 +-
 .../sentry/service/thrift/SentryService.java    | 150 +++---
 .../sentry/service/thrift/ServiceConstants.java |   7 +
 .../apache/sentry/service/thrift/Status.java    |   7 +
 .../main/resources/sentry_common_service.thrift |   1 +
 .../thrift/TestSentryServerWithoutKerberos.java |  45 ++
 .../thrift/TestSentryServiceFailureCase.java    |  26 +-
 .../thrift/TestSentryServiceIntegration.java    |   8 +-
 .../thrift/SentryServiceIntegrationBase.java    |  60 ++-
 .../file/SimpleFileProviderBackend.java         |   4 -
 .../file/TestSimpleFileProvderBackend.java      |   4 +-
 sentry-tests/sentry-tests-hive/pom.xml          |   5 +
 .../apache/sentry/tests/e2e/hive/Context.java   |  18 +-
 .../tests/e2e/hive/TestDatabaseProvider.java    | 124 +++++
 .../e2e/hive/TestPrivilegesAtDatabaseScope.java |   1 -
 .../e2e/hive/hiveserver/HiveServerFactory.java  |   1 +
 32 files changed, 1773 insertions(+), 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ddf75f4..5bda553 100644
--- a/pom.xml
+++ b/pom.xml
@@ -67,8 +67,8 @@ limitations under the License.
     <jdo-api.version>3.0.1</jdo-api.version>
     <derby.version>10.4.2.0</derby.version>
     <commons-cli.version>1.2</commons-cli.version>
-    <hive.version>0.12.0-cdh5.0.0-beta-2-SNAPSHOT</hive.version>
-    <hadoop.version>2.2.0-cdh5.0.0-beta-2-SNAPSHOT</hadoop.version>
+    <hive.version>0.12.0-cdh5.0.0-SNAPSHOT</hive.version>
+    <hadoop.version>2.2.0-cdh5.0.0-SNAPSHOT</hadoop.version>
     <fest.reflect.version>1.4.1</fest.reflect.version>
     <guava.version>11.0.2</guava.version>
     <junit.version>4.9</junit.version>
@@ -79,8 +79,8 @@ limitations under the License.
     <shiro.version>1.2.1</shiro.version>
     <slf4j.version>1.6.1</slf4j.version>
     <solr.version>4.7.0</solr.version>
-    <solr.sentry.handlers.version>4.4.0-cdh5.0.0-beta-2-SNAPSHOT</solr.sentry.handlers.version>
-    <zookeeper.version>3.4.5-cdh5.0.0-beta-2-SNAPSHOT</zookeeper.version>
+    <solr.sentry.handlers.version>4.4.0-cdh5.0.0-SNAPSHOT</solr.sentry.handlers.version>
+    <zookeeper.version>3.4.5-cdh5.0.0-SNAPSHOT</zookeeper.version>
   </properties>
 
   <dependencyManagement>
@@ -471,7 +471,7 @@ limitations under the License.
           <artifactId>build-helper-maven-plugin</artifactId>
           <version>${build.helper.maven.plugin.version}</version>
         </plugin>
-	<plugin>
+        <plugin>
           <groupId>org.datanucleus</groupId>
           <artifactId>datanucleus-maven-plugin</artifactId>
           <version>${datanucleus.maven.plugin.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-binding/sentry-binding-hive/pom.xml
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/pom.xml b/sentry-binding/sentry-binding-hive/pom.xml
index 132f4bc..2ce4d87 100644
--- a/sentry-binding/sentry-binding-hive/pom.xml
+++ b/sentry-binding/sentry-binding-hive/pom.xml
@@ -70,6 +70,11 @@ limitations under the License.
       <groupId>org.apache.sentry</groupId>
       <artifactId>sentry-provider-common</artifactId>
     </dependency>
+    <!-- required for SentryGrantRevokeTask -->
+    <dependency>
+      <groupId>org.apache.sentry</groupId>
+      <artifactId>sentry-provider-db</artifactId>
+    </dependency>
     <dependency>
       <groupId>org.apache.sentry</groupId>
       <artifactId>sentry-provider-file</artifactId>
@@ -89,6 +94,11 @@ limitations under the License.
       <artifactId>hadoop-client</artifactId>
       <version>${hadoop.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-all</artifactId>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java
new file mode 100644
index 0000000..db14b6c
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java
@@ -0,0 +1,34 @@
+/**
+ * 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.hadoop.hive;
+
+import java.util.EnumSet;
+
+import org.apache.hadoop.hive.ql.security.authorization.Privilege.PrivilegeType;
+
+public class SentryHiveConstants {
+  // TODO add INSERT
+  public static final EnumSet<PrivilegeType> ALLOWED_PRIVS = EnumSet.of(PrivilegeType.ALL, PrivilegeType.SELECT);
+
+  public static final String PRIVILEGE_NOT_SUPPORTED = "Sentry does not support privilege: ";
+  public static final String COLUMN_PRIVS_NOT_SUPPORTED = "Sentry users should use views to grant privileges on columns";
+  public static final String PARTITION_PRIVS_NOT_SUPPORTED = "Sentry does not support partition level authorization";
+  public static final String GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT = "Sentry does not allow grant/revoke on: ";
+  public static final String GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL = "Sentry does not allow privileges to be granted/revoked to/from: ";
+  public static final String GRANT_OPTION_NOT_SUPPORTED = "Sentry does not allow WITH GRANT OPTION";
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/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
new file mode 100644
index 0000000..70b05b6
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
@@ -0,0 +1,496 @@
+package org.apache.hadoop.hive.ql.exec;
+/**
+ * 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.
+ */
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.SentryHiveConstants;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.ql.DriverContext;
+import org.apache.hadoop.hive.ql.QueryPlan;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.plan.DDLWork;
+import org.apache.hadoop.hive.ql.plan.GrantDesc;
+import org.apache.hadoop.hive.ql.plan.GrantRevokeRoleDDL;
+import org.apache.hadoop.hive.ql.plan.PrincipalDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeObjectDesc;
+import org.apache.hadoop.hive.ql.plan.RevokeDesc;
+import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
+import org.apache.hadoop.hive.ql.plan.ShowGrantDesc;
+import org.apache.hadoop.hive.ql.plan.api.StageType;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
+import org.apache.sentry.SentryUserException;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
+import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
+import org.apache.sentry.core.common.Subject;
+import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
+import org.apache.sentry.service.thrift.SentryServiceClientFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Iterables;
+
+// TODO remove this suppress
+@SuppressWarnings("unused")
+public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable {
+  private static final Logger LOG = LoggerFactory
+      .getLogger(SentryGrantRevokeTask.class);
+  private static final int RETURN_CODE_SUCCESS = 0;
+  private static final int RETURN_CODE_FAILURE = 1;
+  private static final Splitter DB_TBL_SPLITTER = Splitter.on(".").omitEmptyStrings().trimResults();
+  private static final int separator = Utilities.tabCode;
+  private static final int terminator = Utilities.newLineCode;
+  private static final long serialVersionUID = -7625118066790571999L;
+
+  private SentryServiceClientFactory sentryClientFactory;
+  private SentryPolicyServiceClient sentryClient;
+  private HiveConf conf;
+  private HiveAuthzConf authzConf;
+  private String server;
+  private Subject subject;
+  private Set<String> subjectGroups;
+
+
+  public SentryGrantRevokeTask() {
+    this(new SentryServiceClientFactory());
+  }
+  public SentryGrantRevokeTask(SentryServiceClientFactory sentryClientFactory) {
+    super();
+    this.sentryClientFactory = sentryClientFactory;
+  }
+
+
+  @Override
+  public void initialize(HiveConf conf, QueryPlan queryPlan, DriverContext ctx) {
+    super.initialize(conf, queryPlan, driverContext);
+    this.conf = conf;
+  }
+
+  @Override
+  public int execute(DriverContext driverContext) {
+    try {
+      try {
+        this.sentryClient = sentryClientFactory.create(authzConf);
+      } catch (Exception e) {
+        String msg = "Error creating Sentry client: " + e.getMessage();
+        LOG.error(msg, e);
+        throw new RuntimeException(msg, e);
+      }
+      Preconditions.checkNotNull(authzConf, "HiveAuthConf cannot be null");
+      Preconditions.checkNotNull(subject, "Subject cannot be null");
+      Preconditions.checkNotNull(subjectGroups, "Subject Groups cannot be null");
+      server = Preconditions.checkNotNull(authzConf.get(AuthzConfVars.AUTHZ_SERVER_NAME.getVar()),
+          "Config " + AuthzConfVars.AUTHZ_SERVER_NAME.getVar() + " is required");
+      if (work.getRoleDDLDesc() != null) {
+        return processRoleDDL(conf, console, sentryClient, subject.getName(), subjectGroups,
+            work.getRoleDDLDesc());
+      }
+      if (work.getGrantDesc() != null) {
+        return processGrantDDL(conf, console, sentryClient, subject.getName(), subjectGroups,
+            server, work.getGrantDesc());
+      }
+      if (work.getRevokeDesc() != null) {
+        return processRevokeDDL(conf, console, sentryClient, subject.getName(), subjectGroups,
+            server, work.getRevokeDesc());
+      }
+      if (work.getShowGrantDesc() != null) {
+        return processShowGrantDDL(conf, console, subject.getName(), subjectGroups,
+            work.getShowGrantDesc());
+      }
+      if (work.getGrantRevokeRoleDDL() != null) {
+        return processGrantRevokeRoleDDL(conf, console, sentryClient, subject.getName(), subjectGroups,
+            work.getGrantRevokeRoleDDL());
+      }
+      throw new AssertionError("Unknown command passed to Sentry Grant/Revoke Task");
+    } catch(Throwable throwable) {
+      setException(throwable);
+      String msg = "Error processing Sentry command: " + throwable.getMessage();
+      LOG.error(msg, throwable);
+      console.printError(msg);
+      return RETURN_CODE_FAILURE;
+    } finally {
+      if (sentryClient != null) {
+        sentryClient.close();
+      }
+    }
+  }
+
+  public void setAuthzConf(HiveAuthzConf authzConf) {
+    Preconditions.checkState(this.authzConf == null,
+        "setAuthzConf should only be called once: " + this.authzConf);
+    this.authzConf = authzConf;
+  }
+  public void setSubject(Subject subject) {
+    Preconditions.checkState(this.subject == null,
+        "setSubject should only be called once: " + this.subject);
+    this.subject = subject;
+  }
+  public void setSubjectGroups(Set<String> subjectGroups) {
+    Preconditions.checkState(this.subjectGroups == null,
+        "setSubjectGroups should only be called once: " + this.subjectGroups);
+    this.subjectGroups = subjectGroups;
+  }
+
+  @VisibleForTesting
+  static int processRoleDDL(HiveConf conf, LogHelper console,
+      SentryPolicyServiceClient sentryClient, String subject,
+      Set<String> subjectGroups, RoleDDLDesc desc) throws SentryUserException {
+    RoleDDLDesc.RoleOperation operation = desc.getOperation();
+    DataOutputStream outStream = null;
+    String name = desc.getName();
+    try {
+      if (operation.equals(RoleDDLDesc.RoleOperation.CREATE_ROLE)) {
+        SessionState.get().getAuthenticator();
+        sentryClient.createRole(subject, subjectGroups, name);
+        return RETURN_CODE_SUCCESS;
+      } else if (operation.equals(RoleDDLDesc.RoleOperation.DROP_ROLE)) {
+        sentryClient.dropRole(subject, subjectGroups, name);
+        return RETURN_CODE_SUCCESS;
+      } else if (operation.equals(RoleDDLDesc.RoleOperation.SHOW_ROLE_GRANT)) {
+        PrincipalType principalType = desc.getPrincipalType();
+        if (principalType != PrincipalType.GROUP) {
+          String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + principalType;
+          throw new HiveException(msg);
+        }
+        throw new AssertionError("TODO");
+        // TODO once retrieval API is implemented this can be implemented
+//        List<String> roles = sentryClient.getRoles(name);
+//        if (!roles.isEmpty()) {
+//          Path resFile = new Path(desc.getResFile());
+//          FileSystem fs = resFile.getFileSystem(conf);
+//          outStream = fs.create(resFile);
+//          for (String role : roles) {
+//            outStream.writeBytes("role name:" + role);
+//            outStream.write(terminator);
+//          }
+//          outStream = close(outStream);
+//        }
+//        return RETURN_CODE_SUCCESS;
+      } else {
+        throw new HiveException("Unkown role operation "
+            + operation.getOperationName());
+      }
+    } catch (HiveException e) {
+      String msg = "Error in role operation "
+          + operation.getOperationName() + " on role name "
+          + name + ", error message " + e.getMessage();
+      LOG.warn(msg, e);
+      console.printError(msg);
+      return RETURN_CODE_FAILURE;
+//    } catch (IOException e) {
+//      String msg = "IO Error in role operation " + e.getMessage();
+//      LOG.info(msg, e);
+//      console.printError(msg);
+//      return RETURN_CODE_FAILURE;
+    } finally {
+      closeQuiet(outStream);
+    }
+  }
+
+  @VisibleForTesting
+  static int processGrantDDL(HiveConf conf, LogHelper console,
+      SentryPolicyServiceClient sentryClient, String subject,
+      Set<String> subjectGroups, String server, GrantDesc desc) throws SentryUserException {
+    return processGrantRevokeDDL(console, sentryClient, subject, subjectGroups,
+        server, true, desc.getPrincipals(), desc.getPrivileges(), desc.getPrivilegeSubjectDesc());
+  }
+
+  @VisibleForTesting
+  static int processRevokeDDL(HiveConf conf, LogHelper console,
+      SentryPolicyServiceClient sentryClient, String subject,
+      Set<String> subjectGroups, String server, RevokeDesc desc) throws SentryUserException {
+    return processGrantRevokeDDL(console, sentryClient, subject, subjectGroups,
+        server, false, desc.getPrincipals(), desc.getPrivileges(),
+        desc.getPrivilegeSubjectDesc());
+  }
+
+  @VisibleForTesting
+  static int processShowGrantDDL(HiveConf conf, LogHelper console, String subject,
+      Set<String> subjectGroups, ShowGrantDesc desc) {
+    DataOutputStream outStream = null;
+    try {
+      Path resFile = new Path(desc.getResFile());
+      FileSystem fs = resFile.getFileSystem(conf);
+      outStream = fs.create(resFile);
+      PrincipalDesc principalDesc = desc.getPrincipalDesc();
+      PrivilegeObjectDesc hiveObjectDesc = desc.getHiveObj();
+      String principalName = principalDesc.getName();
+      List<String> columns = desc.getColumns();
+      if (columns != null && !columns.isEmpty()) {
+        throw new HiveException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+      }
+      if (hiveObjectDesc == null) {
+        // TDOD get users from somewhere?
+        List<String> users = Collections.emptyList();
+        if (users != null && users.size() > 0) {
+          boolean first = true;
+          Collections.sort(users);
+          for (String usr : users) {
+            if (!first) {
+              outStream.write(terminator);
+            } else {
+              first = false;
+            }
+            // TODO write grant info
+          }
+        }
+      } else {
+        if (hiveObjectDesc.getPartSpec() != null) {
+          throw new HiveException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
+        }
+        String obj = hiveObjectDesc.getObject();
+        String dbName = null;
+        String tableName = null;
+        if (hiveObjectDesc.getTable()) {
+          DatabaseTable dbTable = parseDBTable(obj);
+          dbName = dbTable.getDatabase();
+          tableName = dbTable.getTable();
+        } else {
+          dbName = hiveObjectDesc.getObject();
+        }
+        if (hiveObjectDesc.getTable()) {
+          // show table level privileges
+          // TODO
+          List<String> tbls = Collections.emptyList();
+          if (tbls != null && tbls.size() > 0) {
+            boolean first = true;
+            Collections.sort(tbls);
+            for (String tbl : tbls) {
+              if (!first) {
+                outStream.write(terminator);
+              } else {
+                first = false;
+              }
+              // TODO write grant info
+            }
+          }
+        } else {
+          // show database level privileges
+          // TODO
+          List<String> dbs = Collections.emptyList();
+          if (dbs != null && dbs.size() > 0) {
+            boolean first = true;
+            Collections.sort(dbs);
+            for (String db : dbs) {
+              if (!first) {
+                outStream.write(terminator);
+              } else {
+                first = false;
+              }
+              // TODO write grant info
+            }
+          }
+        }
+      }
+      outStream = close(outStream);
+    } catch (HiveException e) {
+      String msg = "Error in show grant operation " + e.getMessage();
+      LOG.warn(msg, e);
+      console.printError(msg);
+      return RETURN_CODE_FAILURE;
+    } catch (IOException e) {
+      String msg = "IO Error in show grant " + e.getMessage();
+      LOG.info(msg, e);
+      console.printError(msg);
+      return RETURN_CODE_FAILURE;
+    } finally {
+      closeQuiet(outStream);
+    }
+    return RETURN_CODE_SUCCESS;
+  }
+
+  @VisibleForTesting
+  static int processGrantRevokeRoleDDL(HiveConf conf, LogHelper console,
+      SentryPolicyServiceClient sentryClient, String subject, Set<String> subjectGroups,
+      GrantRevokeRoleDDL desc) throws SentryUserException {
+    try {
+      boolean grantRole = desc.getGrant();
+      List<PrincipalDesc> principals = desc.getPrincipalDesc();
+      List<String> roles = desc.getRoles();
+      for (PrincipalDesc principal : principals) {
+        if (principal.getType() != PrincipalType.GROUP) {
+          String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL +
+              principal.getType();
+          throw new HiveException(msg);
+        }
+        String groupName = principal.getName();
+        for (String roleName : roles) {
+          if (grantRole) {
+            sentryClient.grantRoleToGroup(subject, subjectGroups, groupName, roleName);
+          } else {
+            sentryClient.revokeRoleFromGroup(subject, subjectGroups, groupName, roleName);
+          }
+        }
+      }
+    } catch (HiveException e) {
+      String msg = "Error in grant/revoke operation, error message " + e.getMessage();
+      LOG.warn(msg, e);
+      console.printError(msg);
+      return RETURN_CODE_FAILURE;
+    }
+    return RETURN_CODE_SUCCESS;
+  }
+
+  private static int processGrantRevokeDDL(LogHelper console,
+      SentryPolicyServiceClient sentryClient, String subject,
+      Set<String> subjectGroups, String server,
+      boolean isGrant, List<PrincipalDesc> principals,
+      List<PrivilegeDesc> privileges, PrivilegeObjectDesc privSubjectDesc) throws SentryUserException {
+    if (privileges == null || privileges.size() == 0) {
+      console.printError("No privilege found.");
+      return RETURN_CODE_FAILURE;
+    }
+    String dbName = null;
+    String tableName = null;
+    try {
+      if (privSubjectDesc == null) {
+        throw new HiveException("Privilege subject cannot be null");
+      }
+      if (privSubjectDesc.getPartSpec() != null) {
+        throw new HiveException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
+      }
+      // TODO how to grant all on server
+      String obj = privSubjectDesc.getObject();
+      if (privSubjectDesc.getTable()) {
+        DatabaseTable dbTable = parseDBTable(obj);
+        dbName = dbTable.getDatabase();
+        tableName = dbTable.getTable();
+      } else {
+        dbName = privSubjectDesc.getObject();
+      }
+      for (PrivilegeDesc privDesc : privileges) {
+        List<String> columns = privDesc.getColumns();
+        if (columns != null && !columns.isEmpty()) {
+          throw new HiveException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+        }
+        if (!SentryHiveConstants.ALLOWED_PRIVS.contains(privDesc.getPrivilege().getPriv())) {
+          String msg = SentryHiveConstants.PRIVILEGE_NOT_SUPPORTED + privDesc.getPrivilege().getPriv();
+          throw new HiveException(msg);
+        }
+      }
+      for (PrincipalDesc princ : principals) {
+        if (princ.getType() != PrincipalType.ROLE) {
+          String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + princ.getType();
+          throw new HiveException(msg);
+        }
+        for (PrivilegeDesc privDesc : privileges) {
+          if (isGrant) {
+            if (tableName == null) {
+              sentryClient.grantDatabasePrivilege(subject, subjectGroups, princ.getName(), server, dbName);
+            } else {
+              sentryClient.grantTablePrivilege(subject, subjectGroups, princ.getName(), server, dbName,
+                  tableName, privDesc.getPrivilege().getPriv().name());
+            }
+          } else {
+            if (tableName == null) {
+              sentryClient.revokeDatabasePrivilege(subject, subjectGroups, princ.getName(), server, dbName);
+            } else {
+              sentryClient.revokeTablePrivilege(subject, subjectGroups, princ.getName(), server, dbName,
+                  tableName, privDesc.getPrivilege().getPriv().name());
+            }
+          }
+        }
+      }
+      return RETURN_CODE_SUCCESS;
+    } catch (HiveException e) {
+      String msg = "Error in grant/revoke operation, error message " + e.getMessage();
+      LOG.warn(msg, e);
+      console.printError(msg);
+      return RETURN_CODE_FAILURE;
+    }
+  }
+
+  private static DatabaseTable parseDBTable(String obj) throws HiveException {
+    String[] dbTab = Iterables.toArray(DB_TBL_SPLITTER.split(obj), String.class);
+    if (dbTab.length == 2) {
+      return new DatabaseTable(dbTab[0], dbTab[1]);
+    } else if (dbTab.length == 1){
+      return new DatabaseTable(SessionState.get().getCurrentDatabase(), obj);
+    } else {
+      String msg = "Malformed database.table '" + obj + "'";
+      throw new HiveException(msg);
+    }
+  }
+
+  private static class DatabaseTable {
+    private final String database;
+    private final String table;
+    public DatabaseTable(String database, String table) {
+      this.database = database;
+      this.table = table;
+    }
+    public String getDatabase() {
+      return database;
+    }
+    public String getTable() {
+      return table;
+    }
+  }
+
+  /**
+   * Close to be used in the try block of a try-catch-finally
+   * statement. Returns null so the close/set to null idiom can be
+   * completed in a single line.
+   */
+  private static DataOutputStream close(DataOutputStream out)
+      throws IOException {
+    if (out != null) {
+      out.close();
+    }
+    return null;
+  }
+  /**
+   * Close to be used in the finally block of a try-catch-finally
+   * statement.
+   */
+  private static void closeQuiet(DataOutputStream out) {
+    try {
+      close(out);
+    } catch (IOException e) {
+      LOG.warn("Error closing output stream", e);
+    }
+  }
+
+  @Override
+  public boolean requireLock() {
+    return false;
+  }
+
+  @Override
+  public StageType getType() {
+    return StageType.DDL;
+  }
+
+  @Override
+  public String getName() {
+    return "SENTRY";
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
index 5ff7764..0b83299 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingHook.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hive.ql.HiveDriverFilterHook;
 import org.apache.hadoop.hive.ql.HiveDriverFilterHookContext;
 import org.apache.hadoop.hive.ql.HiveDriverFilterHookResult;
 import org.apache.hadoop.hive.ql.HiveDriverFilterHookResultImpl;
+import org.apache.hadoop.hive.ql.exec.SentryGrantRevokeTask;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.hooks.Entity;
 import org.apache.hadoop.hive.ql.hooks.Entity.Type;
@@ -265,11 +266,23 @@ implements HiveDriverFilterHook {
   @Override
   public void postAnalyze(HiveSemanticAnalyzerHookContext context,
       List<Task<? extends Serializable>> rootTasks) throws SemanticException {
-
     HiveOperation stmtOperation = getCurrentHiveStmtOp();
     HiveAuthzPrivileges stmtAuthObject =
         HiveAuthzPrivilegesMap.getHiveAuthzPrivileges(stmtOperation);
 
+    // must occur above the null check on stmtAuthObject
+    // since GRANT/REVOKE/etc are not authorized by binding layer at present
+    Subject subject = getCurrentSubject(context);
+    Set<String> subjectGroups = hiveAuthzBinding.getGroups(subject);
+    for (Task<? extends Serializable> task : rootTasks) {
+      if (task instanceof SentryGrantRevokeTask) {
+        SentryGrantRevokeTask sentryTask = (SentryGrantRevokeTask)task;
+        sentryTask.setAuthzConf(authzConf);
+        sentryTask.setSubject(subject);
+        sentryTask.setSubjectGroups(subjectGroups);
+      }
+    }
+
     if (stmtAuthObject == null) {
       // We don't handle authorizing this statement
       return;

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingSessionHook.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingSessionHook.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingSessionHook.java
index 1506d8a..2e53acb 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingSessionHook.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/HiveAuthzBindingSessionHook.java
@@ -22,6 +22,8 @@ import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.session.HiveSessionHookContext;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 
+import com.google.common.base.Joiner;
+
 public class HiveAuthzBindingSessionHook
     implements org.apache.hive.service.cli.session.HiveSessionHook {
 
@@ -32,25 +34,26 @@ public class HiveAuthzBindingSessionHook
   public static final String FILTER_HOOK =
     "org.apache.sentry.binding.hive.HiveAuthzBindingHook";
   public static final String SCRATCH_DIR_PERMISSIONS = "700";
-  public static final String ACCESS_RESTRICT_LIST =
-    ConfVars.SEMANTIC_ANALYZER_HOOK.varname + "," +
-    ConfVars.PREEXECHOOKS.varname + "," +
-    ConfVars.HIVE_EXEC_FILTER_HOOK.varname + "," +
-    ConfVars.HIVE_EXTENDED_ENITITY_CAPTURE.varname + "," +
-    ConfVars.SCRATCHDIR.varname + "," +
-    ConfVars.LOCALSCRATCHDIR.varname + "," +
-    ConfVars.METASTOREURIS.varname + "," +
-    ConfVars.METASTORECONNECTURLKEY.varname + "," +
-    ConfVars.HADOOPBIN.varname + "," +
-    ConfVars.HIVESESSIONID.varname + "," +
-    ConfVars.HIVEAUXJARS.varname + "," +
-    ConfVars.HIVESTATSDBCONNECTIONSTRING.varname + "," +
-    ConfVars.SCRATCHDIRPERMISSION.varname + "," +
-    ConfVars.HIVE_SECURITY_COMMAND_WHITELIST.varname + "," +
-    HiveAuthzConf.HIVE_ACCESS_CONF_URL + "," +
-    HiveAuthzConf.HIVE_SENTRY_CONF_URL + "," +
-    HiveAuthzConf.HIVE_ACCESS_SUBJECT_NAME + "," +
-    HiveAuthzConf.HIVE_SENTRY_SUBJECT_NAME;
+  public static final String ACCESS_RESTRICT_LIST = Joiner.on(",").join(
+    ConfVars.SEMANTIC_ANALYZER_HOOK.varname,
+    ConfVars.PREEXECHOOKS.varname,
+    ConfVars.HIVE_EXEC_FILTER_HOOK.varname,
+    ConfVars.HIVE_EXTENDED_ENITITY_CAPTURE.varname,
+    ConfVars.SCRATCHDIR.varname,
+    ConfVars.LOCALSCRATCHDIR.varname,
+    ConfVars.METASTOREURIS.varname,
+    ConfVars.METASTORECONNECTURLKEY.varname,
+    ConfVars.HADOOPBIN.varname,
+    ConfVars.HIVESESSIONID.varname,
+    ConfVars.HIVEAUXJARS.varname,
+    ConfVars.HIVESTATSDBCONNECTIONSTRING.varname,
+    ConfVars.SCRATCHDIRPERMISSION.varname,
+    ConfVars.HIVE_SECURITY_COMMAND_WHITELIST.varname,
+    ConfVars.HIVE_AUTHORIZATION_TASK_FACTORY.varname,
+    HiveAuthzConf.HIVE_ACCESS_CONF_URL,
+    HiveAuthzConf.HIVE_SENTRY_CONF_URL,
+    HiveAuthzConf.HIVE_ACCESS_SUBJECT_NAME,
+    HiveAuthzConf.HIVE_SENTRY_SUBJECT_NAME);
 
   /**
    * The session hook for sentry authorization that sets the required session level configuration

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
new file mode 100644
index 0000000..252d93b
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/sentry/binding/hive/SentryHiveAuthorizationTaskFactoryImpl.java
@@ -0,0 +1,330 @@
+/*
+ * 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.binding.hive;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.SentryHiveConstants;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.ql.exec.SentryGrantRevokeTask;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.exec.TaskFactory;
+import org.apache.hadoop.hive.ql.hooks.ReadEntity;
+import org.apache.hadoop.hive.ql.hooks.WriteEntity;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.HiveParser;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.parse.authorization.HiveAuthorizationTaskFactory;
+import org.apache.hadoop.hive.ql.plan.DDLWork;
+import org.apache.hadoop.hive.ql.plan.GrantDesc;
+import org.apache.hadoop.hive.ql.plan.GrantRevokeRoleDDL;
+import org.apache.hadoop.hive.ql.plan.PrincipalDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeObjectDesc;
+import org.apache.hadoop.hive.ql.plan.RevokeDesc;
+import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
+import org.apache.hadoop.hive.ql.plan.ShowGrantDesc;
+import org.apache.hadoop.hive.ql.security.authorization.Privilege;
+import org.apache.hadoop.hive.ql.security.authorization.PrivilegeRegistry;
+import org.apache.hadoop.hive.ql.session.SessionState;
+
+public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFactory {
+
+
+  public SentryHiveAuthorizationTaskFactoryImpl(HiveConf conf, Hive db) {
+
+  }
+
+  @Override
+  public Task<? extends Serializable> createCreateRoleTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) {
+    String roleName = BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText());
+    RoleDDLDesc roleDesc = new RoleDDLDesc(roleName, RoleDDLDesc.RoleOperation.CREATE_ROLE);
+    return createTask(new DDLWork(inputs, outputs, roleDesc));
+  }
+  @Override
+  public Task<? extends Serializable> createDropRoleTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) {
+    String roleName = BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText());
+    RoleDDLDesc roleDesc = new RoleDDLDesc(roleName, RoleDDLDesc.RoleOperation.DROP_ROLE);
+    return createTask(new DDLWork(inputs, outputs, roleDesc));
+  }
+  @Override
+  public Task<? extends Serializable> createShowRoleGrantTask(ASTNode ast, Path resultFile,
+      HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs) throws SemanticException {
+    ASTNode child = (ASTNode) ast.getChild(0);
+    PrincipalType principalType = PrincipalType.USER;
+    switch (child.getType()) {
+    case HiveParser.TOK_USER:
+      principalType = PrincipalType.USER;
+      break;
+    case HiveParser.TOK_GROUP:
+      principalType = PrincipalType.GROUP;
+      break;
+    case HiveParser.TOK_ROLE:
+      principalType = PrincipalType.ROLE;
+      break;
+    }
+    if (principalType != PrincipalType.GROUP) {
+      String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + principalType;
+      throw new SemanticException(msg);
+    }
+    String principalName = BaseSemanticAnalyzer.unescapeIdentifier(child.getChild(0).getText());
+    RoleDDLDesc roleDesc = new RoleDDLDesc(principalName, principalType,
+        RoleDDLDesc.RoleOperation.SHOW_ROLE_GRANT, null);
+    roleDesc.setResFile(resultFile.toString());
+    return createTask(new DDLWork(inputs, outputs,  roleDesc));
+  }
+
+  @Override
+  public Task<? extends Serializable> createGrantTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException {
+    List<PrivilegeDesc> privilegeDesc = analyzePrivilegeListDef(
+        (ASTNode) ast.getChild(0));
+    List<PrincipalDesc> principalDesc = analyzePrincipalListDef(
+        (ASTNode) ast.getChild(1));
+    PrivilegeObjectDesc privilegeObj = null;
+
+    if (ast.getChildCount() > 2) {
+      for (int i = 2; i < ast.getChildCount(); i++) {
+        ASTNode astChild = (ASTNode) ast.getChild(i);
+        if (astChild.getType() == HiveParser.TOK_GRANT_WITH_OPTION) {
+          throw new SemanticException(SentryHiveConstants.GRANT_OPTION_NOT_SUPPORTED);
+        } else if (astChild.getType() == HiveParser.TOK_PRIV_OBJECT) {
+          privilegeObj = analyzePrivilegeObject(astChild);
+        }
+      }
+    }
+    String userName = null;
+    if (SessionState.get() != null
+        && SessionState.get().getAuthenticator() != null) {
+      userName = SessionState.get().getAuthenticator().getUserName();
+    }
+    if (privilegeObj.getPartSpec() != null) {
+      throw new SemanticException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
+    }
+    for (PrivilegeDesc privDesc : privilegeDesc) {
+      List<String> columns = privDesc.getColumns();
+      if (columns != null && !columns.isEmpty()) {
+        throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+      }
+    }
+    for (PrincipalDesc princ : principalDesc) {
+      if (princ.getType() != PrincipalType.ROLE) {
+        String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + princ.getType();
+        throw new SemanticException(msg);
+      }
+    }
+    GrantDesc grantDesc = new GrantDesc(privilegeObj, privilegeDesc,
+        principalDesc, userName, PrincipalType.USER, false);
+    return createTask(new DDLWork(inputs, outputs, grantDesc));
+  }
+  @Override
+  public Task<? extends Serializable> createRevokeTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException {
+    List<PrivilegeDesc> privilegeDesc = analyzePrivilegeListDef((ASTNode) ast.getChild(0));
+    List<PrincipalDesc> principalDesc = analyzePrincipalListDef((ASTNode) ast.getChild(1));
+    PrivilegeObjectDesc privilegeObj = null;
+    if (ast.getChildCount() > 2) {
+      ASTNode astChild = (ASTNode) ast.getChild(2);
+      privilegeObj = analyzePrivilegeObject(astChild);
+    }
+    if (privilegeObj.getPartSpec() != null) {
+      throw new SemanticException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
+    }
+    for (PrivilegeDesc privDesc : privilegeDesc) {
+      List<String> columns = privDesc.getColumns();
+      if (columns != null && !columns.isEmpty()) {
+        throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+      }
+    }
+    for (PrincipalDesc princ : principalDesc) {
+      if (princ.getType() != PrincipalType.ROLE) {
+        String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + princ.getType();
+        throw new SemanticException(msg);
+      }
+    }
+    RevokeDesc revokeDesc = new RevokeDesc(privilegeDesc, principalDesc, privilegeObj);
+    return createTask(new DDLWork(inputs, outputs, revokeDesc));
+  }
+
+  @Override
+  public Task<? extends Serializable> createGrantRoleTask(ASTNode ast, HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs)
+      throws SemanticException {
+    return analyzeGrantRevokeRole(true, ast, inputs, outputs);
+  }
+
+  @Override
+  public Task<? extends Serializable> createShowGrantTask(ASTNode ast, Path resultFile, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException {
+    PrivilegeObjectDesc privHiveObj = null;
+
+    ASTNode principal = (ASTNode) ast.getChild(0);
+    PrincipalType type = PrincipalType.USER;
+    switch (principal.getType()) {
+    case HiveParser.TOK_USER:
+      type = PrincipalType.USER;
+      break;
+    case HiveParser.TOK_GROUP:
+      type = PrincipalType.GROUP;
+      break;
+    case HiveParser.TOK_ROLE:
+      type = PrincipalType.ROLE;
+      break;
+    }
+    if (type != PrincipalType.ROLE) {
+      String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + type;
+      throw new SemanticException(msg);
+    }
+    String principalName = BaseSemanticAnalyzer.unescapeIdentifier(principal.getChild(0).getText());
+    PrincipalDesc principalDesc = new PrincipalDesc(principalName, type);
+    if (ast.getChildCount() > 1) {
+      ASTNode child = (ASTNode) ast.getChild(1);
+      if (child.getToken().getType() == HiveParser.TOK_PRIV_OBJECT_COL) {
+        privHiveObj = new PrivilegeObjectDesc();
+        privHiveObj.setObject(BaseSemanticAnalyzer.unescapeIdentifier(child.getChild(0).getText()));
+        if (child.getChildCount() > 1) {
+          for (int i = 1; i < child.getChildCount(); i++) {
+            ASTNode grandChild = (ASTNode) child.getChild(i);
+            if (grandChild.getToken().getType() == HiveParser.TOK_PARTSPEC) {
+              throw new SemanticException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
+            } else if (grandChild.getToken().getType() == HiveParser.TOK_TABCOLNAME) {
+              throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+            } else {
+              privHiveObj.setTable(child.getChild(i) != null);
+            }
+          }
+        }
+      }
+    }
+
+    if (privHiveObj == null) {
+      throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+    }
+
+    ShowGrantDesc showGrant = new ShowGrantDesc(resultFile.toString(),
+        principalDesc, privHiveObj, null);
+    return createTask(new DDLWork(inputs, outputs, showGrant));
+  }
+
+  @Override
+  public Task<? extends Serializable> createRevokeRoleTask(ASTNode ast, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) throws SemanticException {
+    return analyzeGrantRevokeRole(false, ast, inputs, outputs);
+  }
+
+  private Task<? extends Serializable> analyzeGrantRevokeRole(boolean isGrant, ASTNode ast,
+      HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs) throws SemanticException {
+    List<PrincipalDesc> principalDesc = analyzePrincipalListDef(
+        (ASTNode) ast.getChild(0));
+    List<String> roles = new ArrayList<String>();
+    for (int i = 1; i < ast.getChildCount(); i++) {
+      roles.add(BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(i).getText()));
+    }
+    String roleOwnerName = "";
+    if (SessionState.get() != null
+        && SessionState.get().getAuthenticator() != null) {
+      roleOwnerName = SessionState.get().getAuthenticator().getUserName();
+    }
+    for (PrincipalDesc princ : principalDesc) {
+      if (princ.getType() != PrincipalType.GROUP) {
+        String msg = SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT + princ.getType();
+        throw new SemanticException(msg);
+      }
+    }
+    GrantRevokeRoleDDL grantRevokeRoleDDL = new GrantRevokeRoleDDL(isGrant,
+        roles, principalDesc, roleOwnerName, PrincipalType.USER, false);
+    return createTask(new DDLWork(inputs, outputs, grantRevokeRoleDDL));
+  }
+
+  private PrivilegeObjectDesc analyzePrivilegeObject(ASTNode ast)
+      throws SemanticException {
+    PrivilegeObjectDesc subject = new PrivilegeObjectDesc();
+    subject.setObject(BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText()));
+    if (ast.getChildCount() > 1) {
+      for (int i = 0; i < ast.getChildCount(); i++) {
+        ASTNode astChild = (ASTNode) ast.getChild(i);
+        if (astChild.getToken().getType() == HiveParser.TOK_PARTSPEC) {
+          throw new SemanticException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
+        } else {
+          subject.setTable(ast.getChild(0) != null);
+        }
+      }
+    }
+    return subject;
+  }
+
+  private List<PrincipalDesc> analyzePrincipalListDef(ASTNode node) {
+    List<PrincipalDesc> principalList = new ArrayList<PrincipalDesc>();
+    for (int i = 0; i < node.getChildCount(); i++) {
+      ASTNode child = (ASTNode) node.getChild(i);
+      PrincipalType type = null;
+      switch (child.getType()) {
+      case HiveParser.TOK_USER:
+        type = PrincipalType.USER;
+        break;
+      case HiveParser.TOK_GROUP:
+        type = PrincipalType.GROUP;
+        break;
+      case HiveParser.TOK_ROLE:
+        type = PrincipalType.ROLE;
+        break;
+      }
+      String principalName = BaseSemanticAnalyzer.unescapeIdentifier(child.getChild(0).getText());
+      PrincipalDesc principalDesc = new PrincipalDesc(principalName, type);
+      principalList.add(principalDesc);
+    }
+    return principalList;
+  }
+
+  private List<PrivilegeDesc> analyzePrivilegeListDef(ASTNode node)
+      throws SemanticException {
+    List<PrivilegeDesc> ret = new ArrayList<PrivilegeDesc>();
+    for (int i = 0; i < node.getChildCount(); i++) {
+      ASTNode privilegeDef = (ASTNode) node.getChild(i);
+      ASTNode privilegeType = (ASTNode) privilegeDef.getChild(0);
+      Privilege privObj = PrivilegeRegistry.getPrivilege(privilegeType.getType());
+      if (privObj == null) {
+        throw new SemanticException("undefined privilege " + privilegeType.getType());
+      }
+      if (!SentryHiveConstants.ALLOWED_PRIVS.contains(privObj.getPriv())) {
+        String msg = SentryHiveConstants.PRIVILEGE_NOT_SUPPORTED + privObj.getPriv();
+        throw new SemanticException(msg);
+      }
+      if (privilegeDef.getChildCount() > 1) {
+        throw new SemanticException(SentryHiveConstants.COLUMN_PRIVS_NOT_SUPPORTED);
+      }
+      PrivilegeDesc privilegeDesc = new PrivilegeDesc(privObj, null);
+      ret.add(privilegeDesc);
+    }
+    return ret;
+  }
+
+  private static Task<? extends Serializable> createTask(DDLWork work) {
+    SentryGrantRevokeTask task = new SentryGrantRevokeTask();
+    task.setId("Stage-" + Integer.toString(TaskFactory.getAndIncrementId()));
+    task.setWork(work);
+    return task;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/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 3be0d69..eddf3ae 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
@@ -20,6 +20,7 @@ import java.lang.reflect.Constructor;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -146,10 +147,10 @@ public class HiveAuthzBinding {
         + policyEngineName + ", provider backend " + providerBackendName);
       // load the provider backend class
       Constructor<?> providerBackendConstructor =
-        Class.forName(providerBackendName).getDeclaredConstructor(String.class);
+        Class.forName(providerBackendName).getDeclaredConstructor(Configuration.class, String.class);
       providerBackendConstructor.setAccessible(true);
     ProviderBackend providerBackend = (ProviderBackend) providerBackendConstructor.
-        newInstance(new Object[] {resourceName});
+        newInstance(new Object[] {authzConf, resourceName});
 
     // load the policy engine class
     Constructor<?> policyConstructor =
@@ -235,6 +236,10 @@ public class HiveAuthzBinding {
       }
   }
 
+  public Set<String> getGroups(Subject subject) {
+    return authProvider.getGroupMapping().getGroups(subject.getName());
+  }
+
   public Server getAuthServer() {
     if (!open) {
       throw new IllegalStateException("Binding has been closed");

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestSentryHiveAuthorizationTaskFactory.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestSentryHiveAuthorizationTaskFactory.java b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestSentryHiveAuthorizationTaskFactory.java
new file mode 100644
index 0000000..817537d
--- /dev/null
+++ b/sentry-binding/sentry-binding-hive/src/test/java/org/apache/sentry/binding/hive/TestSentryHiveAuthorizationTaskFactory.java
@@ -0,0 +1,374 @@
+/**
+ * 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.binding.hive;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hive.SentryHiveConstants;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.parse.ASTNode;
+import org.apache.hadoop.hive.ql.parse.DDLSemanticAnalyzer;
+import org.apache.hadoop.hive.ql.parse.ParseDriver;
+import org.apache.hadoop.hive.ql.parse.ParseUtils;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.plan.DDLWork;
+import org.apache.hadoop.hive.ql.plan.GrantDesc;
+import org.apache.hadoop.hive.ql.plan.GrantRevokeRoleDDL;
+import org.apache.hadoop.hive.ql.plan.PrincipalDesc;
+import org.apache.hadoop.hive.ql.plan.PrivilegeDesc;
+import org.apache.hadoop.hive.ql.plan.RevokeDesc;
+import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
+import org.apache.hadoop.hive.ql.plan.RoleDDLDesc.RoleOperation;
+import org.apache.hadoop.hive.ql.plan.ShowGrantDesc;
+import org.apache.hadoop.hive.ql.security.HadoopDefaultAuthenticator;
+import org.apache.hadoop.hive.ql.security.authorization.Privilege;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class TestSentryHiveAuthorizationTaskFactory {
+
+  private static final String SELECT = "SELECT";
+  private static final String DB = "default";
+  private static final String TABLE = "table1";
+  private static final String GROUP = "group1";
+  private static final String ROLE = "role1";
+  private static final String USER = "user1";
+
+  private ParseDriver parseDriver;
+  private DDLSemanticAnalyzer analyzer;
+  private HiveConf conf;
+  private Context context;
+  private String currentUser;
+  private Hive db;
+  private Table table;
+  private Partition partition;
+
+  @Before
+  public void setup() throws Exception {
+    conf = new HiveConf();
+    conf.setVar(ConfVars.HIVE_AUTHORIZATION_TASK_FACTORY,
+        SentryHiveAuthorizationTaskFactoryImpl.class.getName());
+    db = Mockito.mock(Hive.class);
+    table = new Table(DB, TABLE);
+    partition = new Partition(table);
+    context = new Context(conf);
+    parseDriver = new ParseDriver();
+    analyzer = new DDLSemanticAnalyzer(conf, db);
+    SessionState.start(conf);
+    Mockito.when(db.getTable(TABLE, false)).thenReturn(table);
+    Mockito.when(db.getPartition(table, new HashMap<String, String>(), false))
+    .thenReturn(partition);
+
+    HadoopDefaultAuthenticator auth = new HadoopDefaultAuthenticator();
+    auth.setConf(conf);
+    currentUser = auth.getUserName();
+
+  }
+
+  /**
+   * CREATE ROLE ...
+   */
+  @Test
+  public void testCreateRole() throws Exception {
+    DDLWork work = analyze(parse("CREATE ROLE " + ROLE));
+    RoleDDLDesc roleDesc = work.getRoleDDLDesc();
+    Assert.assertNotNull("Role should not be null", roleDesc);
+    Assert.assertEquals(RoleOperation.CREATE_ROLE, roleDesc.getOperation());
+    Assert.assertFalse("Did not expect a group", roleDesc.getGroup());
+    Assert.assertEquals(ROLE, roleDesc.getName());
+  }
+
+  /**
+   * DROP ROLE ...
+   */
+  @Test
+  public void testDropRole() throws Exception {
+    DDLWork work = analyze(parse("DROP ROLE " + ROLE));
+    RoleDDLDesc roleDesc = work.getRoleDDLDesc();
+    Assert.assertNotNull("Role should not be null", roleDesc);
+    Assert.assertEquals(RoleOperation.DROP_ROLE, roleDesc.getOperation());
+    Assert.assertFalse("Did not expect a group", roleDesc.getGroup());
+    Assert.assertEquals(ROLE, roleDesc.getName());
+  }
+
+  /**
+   * GRANT ... ON TABLE ... TO USER ...
+   */
+  @Test
+  public void testGrantUserTable() throws Exception {
+    expectSemanticException("GRANT " + SELECT + " ON TABLE " + TABLE + " TO USER " + USER,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "USER");
+  }
+
+  /**
+   * GRANT ... ON TABLE ... TO ROLE ...
+   */
+  @Test
+  public void testGrantRoleTable() throws Exception {
+    DDLWork work = analyze(parse("GRANT " + SELECT + " ON TABLE " + TABLE
+        + " TO ROLE " + ROLE));
+    GrantDesc grantDesc = work.getGrantDesc();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    for (PrincipalDesc principal : assertSize(1, grantDesc.getPrincipals())) {
+      Assert.assertEquals(PrincipalType.ROLE, principal.getType());
+      Assert.assertEquals(ROLE, principal.getName());
+    }
+    for (PrivilegeDesc privilege : assertSize(1, grantDesc.getPrivileges())) {
+      Assert.assertEquals(Privilege.SELECT, privilege.getPrivilege());
+    }
+    Assert.assertTrue("Expected table", grantDesc.getPrivilegeSubjectDesc()
+        .getTable());
+    Assert.assertEquals(TABLE, grantDesc.getPrivilegeSubjectDesc().getObject());
+  }
+  /**
+   * GRANT ... ON TABLE ... TO ROLE ... WITH GRANT OPTION
+   */
+  @Test
+  public void testGrantRoleTableWithGrantOption() throws Exception {
+    expectSemanticException("GRANT " + SELECT + " ON TABLE " + TABLE + " TO ROLE " + ROLE +
+        " WITH GRANT OPTION", "Sentry does not allow WITH GRANT OPTION");
+  }
+
+  /**
+   * GRANT ... ON TABLE ... TO GROUP ...
+   */
+  @Test
+  public void testGrantGroupTable() throws Exception {
+    expectSemanticException("GRANT " + SELECT + " ON TABLE " + TABLE + " TO GROUP " + GROUP,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "GROUP");
+  }
+
+  /**
+   * REVOKE ... ON TABLE ... FROM USER ...
+   */
+  @Test
+  public void testRevokeUserTable() throws Exception {
+    expectSemanticException("REVOKE " + SELECT + " ON TABLE " + TABLE + " FROM USER " + USER,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "USER");
+  }
+
+  /**
+   * REVOKE ... ON TABLE ... FROM ROLE ...
+   */
+  @Test
+  public void testRevokeRoleTable() throws Exception {
+    DDLWork work = analyze(parse("REVOKE " + SELECT + " ON TABLE " + TABLE
+        + " FROM ROLE " + ROLE));
+    RevokeDesc grantDesc = work.getRevokeDesc();
+    Assert.assertNotNull("Revoke should not be null", grantDesc);
+    for (PrincipalDesc principal : assertSize(1, grantDesc.getPrincipals())) {
+      Assert.assertEquals(PrincipalType.ROLE, principal.getType());
+      Assert.assertEquals(ROLE, principal.getName());
+    }
+    for (PrivilegeDesc privilege : assertSize(1, grantDesc.getPrivileges())) {
+      Assert.assertEquals(Privilege.SELECT, privilege.getPrivilege());
+    }
+    Assert.assertTrue("Expected table", grantDesc.getPrivilegeSubjectDesc()
+        .getTable());
+    Assert.assertEquals(TABLE, grantDesc.getPrivilegeSubjectDesc().getObject());
+  }
+
+  /**
+   * REVOKE ... ON TABLE ... FROM GROUP ...
+   */
+  @Test
+  public void testRevokeGroupTable() throws Exception {
+    expectSemanticException("REVOKE " + SELECT + " ON TABLE " + TABLE + " FROM GROUP " + GROUP,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "GROUP");
+  }
+
+  /**
+   * GRANT ROLE ... TO USER ...
+   */
+  @Test
+  public void testGrantRoleUser() throws Exception {
+    expectSemanticException("GRANT ROLE " + ROLE + " TO USER " + USER,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT + "USER");
+  }
+
+  /**
+   * GRANT ROLE ... TO ROLE ...
+   */
+  @Test
+  public void testGrantRoleRole() throws Exception {
+    expectSemanticException("GRANT ROLE " + ROLE + " TO ROLE " + ROLE,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT + "ROLE");
+  }
+
+  /**
+   * GRANT ROLE ... TO GROUP ...
+   */
+  @Test
+  public void testGrantRoleGroup() throws Exception {
+    DDLWork work = analyze(parse("GRANT ROLE " + ROLE + " TO GROUP " + GROUP));
+    GrantRevokeRoleDDL grantDesc = work.getGrantRevokeRoleDDL();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    Assert.assertTrue("Expected grant ", grantDesc.getGrant());
+    Assert
+    .assertFalse("Grant option should be false", grantDesc.isGrantOption());
+    Assert.assertEquals(currentUser, grantDesc.getGrantor());
+    Assert.assertEquals(PrincipalType.USER, grantDesc.getGrantorType());
+    for (String role : assertSize(1, grantDesc.getRoles())) {
+      Assert.assertEquals(ROLE, role);
+    }
+    for (PrincipalDesc principal : assertSize(1, grantDesc.getPrincipalDesc())) {
+      Assert.assertEquals(PrincipalType.GROUP, principal.getType());
+      Assert.assertEquals(GROUP, principal.getName());
+    }
+  }
+
+  /**
+   * REVOKE ROLE ... FROM USER ...
+   */
+  @Test
+  public void testRevokeRoleUser() throws Exception {
+    expectSemanticException("REVOKE ROLE " + ROLE + " FROM USER " + USER,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT + "USER");
+  }
+
+  /**
+   * REVOKE ROLE ... FROM ROLE ...
+   */
+  @Test
+  public void testRevokeRoleRole() throws Exception {
+    expectSemanticException("REVOKE ROLE " + ROLE + " FROM ROLE " + ROLE,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT + "ROLE");
+  }
+
+  /**
+   * REVOKE ROLE ... FROM GROUP ...
+   */
+  @Test
+  public void testRevokeRoleGroup() throws Exception {
+    DDLWork work = analyze(parse("REVOKE ROLE " + ROLE + " FROM GROUP " + GROUP));
+    GrantRevokeRoleDDL grantDesc = work.getGrantRevokeRoleDDL();
+    Assert.assertNotNull("Grant should not be null", grantDesc);
+    Assert.assertFalse("Did not expect grant ", grantDesc.getGrant());
+    Assert.assertFalse("Grant option is always true ", grantDesc.isGrantOption());
+    Assert.assertEquals(currentUser, grantDesc.getGrantor());
+    Assert.assertEquals(PrincipalType.USER, grantDesc.getGrantorType());
+    for (String role : assertSize(1, grantDesc.getRoles())) {
+      Assert.assertEquals(ROLE, role);
+    }
+    for (PrincipalDesc principal : assertSize(1, grantDesc.getPrincipalDesc())) {
+      Assert.assertEquals(PrincipalType.GROUP, principal.getType());
+      Assert.assertEquals(GROUP, principal.getName());
+    }
+  }
+
+  /**
+   * SHOW ROLE GRANT USER ...
+   */
+  @Test
+  public void testShowRoleGrantUser() throws Exception {
+    expectSemanticException("SHOW ROLE GRANT USER " + USER,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "USER");
+  }
+
+  /**
+   * SHOW ROLE GRANT ROLE ...
+   */
+  @Test
+  public void testShowRoleGrantRole() throws Exception {
+    expectSemanticException("SHOW ROLE GRANT ROLE " + ROLE,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "ROLE");
+  }
+
+  /**
+   * SHOW ROLE GRANT GROUP ...
+   */
+  @Test
+  public void testShowRoleGrantGroup() throws Exception {
+    DDLWork work = analyze(parse("SHOW ROLE GRANT GROUP " + GROUP));
+    RoleDDLDesc roleDesc = work.getRoleDDLDesc();
+    Assert.assertNotNull("Role should not be null", roleDesc);
+    Assert.assertEquals(RoleOperation.SHOW_ROLE_GRANT, roleDesc.getOperation());
+    Assert.assertEquals(PrincipalType.GROUP, roleDesc.getPrincipalType());
+    Assert.assertEquals(GROUP, roleDesc.getName());
+  }
+
+  /**
+   * SHOW GRANT USER ... ON TABLE ...
+   */
+  @Test
+  public void testShowGrantUserOnTable() throws Exception {
+    expectSemanticException("SHOW GRANT USER " + USER + " ON TABLE " + TABLE,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "USER");
+  }
+
+  /**
+   * SHOW GRANT ROLE ... ON TABLE ...
+   */
+  @Test
+  public void testShowGrantRoleOnTable() throws Exception {
+    DDLWork work = analyze(parse("SHOW GRANT ROLE " + ROLE + " ON TABLE " + TABLE));
+    ShowGrantDesc grantDesc = work.getShowGrantDesc();
+    Assert.assertNotNull("Show grant should not be null", grantDesc);
+    Assert.assertEquals(PrincipalType.ROLE, grantDesc.getPrincipalDesc().getType());
+    Assert.assertEquals(ROLE, grantDesc.getPrincipalDesc().getName());
+    Assert.assertTrue("Expected table", grantDesc.getHiveObj().getTable());
+    Assert.assertEquals(TABLE, grantDesc.getHiveObj().getObject());
+    Assert.assertTrue("Expected table", grantDesc.getHiveObj().getTable());
+  }
+
+  /**
+   * SHOW GRANT GROUP ... ON TABLE ...
+   */
+  @Test
+  public void testShowGrantGroupOnTable() throws Exception {
+    expectSemanticException("SHOW GRANT GROUP " + GROUP + " ON TABLE " + TABLE,
+        SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "GROUP");
+  }
+
+  private void expectSemanticException(String command, String msg) throws Exception {
+    try {
+      analyze(parse(command));
+      Assert.fail("Expected command '" + command + "' to fail with '" + msg + "'");
+    } catch (SemanticException e) {
+      Assert.assertEquals(msg, e.getMessage());
+    }
+  }
+
+  private ASTNode parse(String command) throws Exception {
+    return ParseUtils.findRootNonNullToken(parseDriver.parse(command));
+  }
+
+  private DDLWork analyze(ASTNode ast) throws Exception {
+    analyzer.analyze(ast, context);
+    List<Task<? extends Serializable>> rootTasks = analyzer.getRootTasks();
+    return (DDLWork) assertSize(1, rootTasks).get(0).getWork();
+  }
+
+  private static <L extends List<?>> L assertSize(int size, L list) {
+    Assert.assertEquals(list.toString(), size, list.size());
+    return list;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java
index 661deff..9dc63e4 100644
--- a/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java
+++ b/sentry-policy/sentry-policy-db/src/test/java/org/apache/sentry/policy/db/DBPolicyFileBackend.java
@@ -18,10 +18,11 @@ package org.apache.sentry.policy.db;
 
 import java.io.IOException;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.sentry.provider.file.SimpleFileProviderBackend;
 
 public class DBPolicyFileBackend extends SimpleDBPolicyEngine {
   public DBPolicyFileBackend(String server, String resource) throws IOException{
-    super(server, new SimpleFileProviderBackend(resource));
+    super(server, new SimpleFileProviderBackend(new Configuration(), resource));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java
----------------------------------------------------------------------
diff --git a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java
index fd8af78..be23e15 100644
--- a/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java
+++ b/sentry-policy/sentry-policy-search/src/test/java/org/apache/sentry/policy/search/SearchPolicyFileBackend.java
@@ -18,10 +18,11 @@ package org.apache.sentry.policy.search;
 
 import java.io.IOException;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.sentry.provider.file.SimpleFileProviderBackend;
 
 public class SearchPolicyFileBackend extends SimpleSearchPolicyEngine {
   public SearchPolicyFileBackend(String resource) throws IOException{
-    super(new SimpleFileProviderBackend(resource));
+    super(new SimpleFileProviderBackend(new Configuration(), resource));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/service/thrift/sentry_common_serviceConstants.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/service/thrift/sentry_common_serviceConstants.java b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/service/thrift/sentry_common_serviceConstants.java
index c465737..4fdeaeb 100644
--- a/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/service/thrift/sentry_common_serviceConstants.java
+++ b/sentry-provider/sentry-provider-db/src/gen/thrift/gen-javabean/org/apache/sentry/service/thrift/sentry_common_serviceConstants.java
@@ -45,4 +45,6 @@ public class sentry_common_serviceConstants {
 
   public static final int TSENTRY_STATUS_INVALID_INPUT = 4;
 
+  public static final int TSENTRY_STATUS_ACCESS_DENIED = 5;
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SentryAccessDeniedException.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SentryAccessDeniedException.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SentryAccessDeniedException.java
new file mode 100644
index 0000000..8f1fa2b
--- /dev/null
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/SentryAccessDeniedException.java
@@ -0,0 +1,27 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.sentry.provider.db;
+
+import org.apache.sentry.SentryUserException;
+
+public class SentryAccessDeniedException extends SentryUserException {
+  private static final long serialVersionUID = 2962080655835L;
+  public SentryAccessDeniedException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/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 bc4d7b5..b068aca 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
@@ -42,10 +42,6 @@ public class SimpleDBProviderBackend implements ProviderBackend {
 
   private volatile boolean initialized;
 
-  public SimpleDBProviderBackend(String resourcePath) throws IOException {
-    this(new Configuration(), new Path(resourcePath));
-  }
-
   public SimpleDBProviderBackend(Configuration conf, String resourcePath) throws IOException {
     this(conf, new Path(resourcePath));
   }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
index 84d9d8d..464569c 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyServiceClient.java
@@ -48,9 +48,9 @@ import com.google.common.collect.Sets;
 
 public class SentryPolicyServiceClient {
 
-  @SuppressWarnings("unused")
   private final Configuration conf;
   private final InetSocketAddress serverAddress;
+  private final boolean kerberos;
   private final String[] serverPrincipalParts;
   private SentryPolicyService.Client client;
   private TTransport transport;
@@ -60,31 +60,38 @@ public class SentryPolicyServiceClient {
 
   public SentryPolicyServiceClient(Configuration conf) throws IOException {
     this.conf = conf;
+    Preconditions.checkNotNull(this.conf, "Configuration object cannot be null");
     this.serverAddress = NetUtils.createSocketAddr(Preconditions.checkNotNull(
                            conf.get(ClientConfig.SERVER_RPC_ADDRESS), "Config key "
                            + ClientConfig.SERVER_RPC_ADDRESS + " is required"), conf.getInt(
                            ClientConfig.SERVER_RPC_PORT, ClientConfig.SERVER_RPC_PORT_DEFAULT));
     this.connectionTimeout = conf.getInt(ClientConfig.SERVER_RPC_CONN_TIMEOUT,
                                          ClientConfig.SERVER_RPC_CONN_TIMEOUT_DEFAULT);
-    String serverPrincipal = Preconditions.checkNotNull(
-                               conf.get(ServerConfig.PRINCIPAL), ServerConfig.PRINCIPAL
-                               + " is required");
-    serverPrincipalParts = SaslRpcServer.splitKerberosName(serverPrincipal);
-    Preconditions.checkArgument(serverPrincipalParts.length == 3,
-                                "Kerberos principal should have 3 parts: " + serverPrincipal);
+    kerberos = ServerConfig.SECURITY_MODE_KERBEROS.equalsIgnoreCase(
+        conf.get(ServerConfig.SECURITY_MODE, ServerConfig.SECURITY_MODE_KERBEROS).trim());
     transport = new TSocket(serverAddress.getHostString(),
-                            serverAddress.getPort(), connectionTimeout);
-    TTransport saslTransport = new TSaslClientTransport(
-      AuthMethod.KERBEROS.getMechanismName(), null, serverPrincipalParts[0],
-      serverPrincipalParts[1], ClientConfig.SASL_PROPERTIES, null, transport);
+        serverAddress.getPort(), connectionTimeout);
+    if (kerberos) {
+      String serverPrincipal = Preconditions.checkNotNull(
+          conf.get(ServerConfig.PRINCIPAL), ServerConfig.PRINCIPAL
+          + " is required");
+      serverPrincipalParts = SaslRpcServer.splitKerberosName(serverPrincipal);
+      Preconditions.checkArgument(serverPrincipalParts.length == 3,
+           "Kerberos principal should have 3 parts: " + serverPrincipal);
+      transport = new TSaslClientTransport(
+          AuthMethod.KERBEROS.getMechanismName(), null, serverPrincipalParts[0],
+          serverPrincipalParts[1], ClientConfig.SASL_PROPERTIES, null, transport);
+    } else {
+      serverPrincipalParts = null;
+    }
     try {
-      saslTransport.open();
+      transport.open();
     } catch (TTransportException e) {
       throw new IOException("Transport exception while opening transport: " + e.getMessage(), e);
     }
-    LOGGER.info("Successfully opened transport");
+    LOGGER.info("Successfully opened transport: " + transport + " to " + serverAddress);
     TMultiplexedProtocol protocol = new TMultiplexedProtocol(
-      new TBinaryProtocol(saslTransport),
+      new TBinaryProtocol(transport),
       SentryPolicyStoreProcessor.SENTRY_POLICY_SERVICE_NAME);
     client = new SentryPolicyService.Client(protocol);
     LOGGER.info("Successfully created client");
@@ -185,6 +192,7 @@ public class SentryPolicyServiceClient {
     privilege.setServerName(serverName);
     privilege.setURI(uri);
     privilege.setDbName(db);
+    privilege.setTableName(table);
     privilege.setAction(action);
     privilege.setGrantorPrincipal(requestorUserName);
     privilege.setCreateTime(System.currentTimeMillis());
@@ -283,6 +291,21 @@ public class SentryPolicyServiceClient {
     }
   }
 
+  public void revokeRoleFromGroup(String requestorUserName, Set<String> requestorUserGroupName,
+      String groupName, String roleName)
+  throws SentryUserException {
+    TAlterSentryRoleDeleteGroupsRequest request = new TAlterSentryRoleDeleteGroupsRequest(ThriftConstants.
+        TSENTRY_SERVICE_VERSION_CURRENT, requestorUserName, requestorUserGroupName,
+        roleName, Sets.newHashSet(new TSentryGroup(groupName)));
+    try {
+      TAlterSentryRoleDeleteGroupsResponse response = client.alter_sentry_role_delete_groups(request);
+      Status.throwIfNotOk(response.getStatus());
+    } catch (TException e) {
+      String msg = "Thrift exception occured: " + e.getMessage();
+      throw new SentryUserException(msg, e);
+    }
+  }
+
   public void close() {
     if (transport != null) {
       transport.close();

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/63c134f3/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
----------------------------------------------------------------------
diff --git a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
index 722b490..9562783 100644
--- a/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
+++ b/sentry-provider/sentry-provider-db/src/main/java/org/apache/sentry/provider/db/service/thrift/SentryPolicyStoreProcessor.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.sentry.provider.db.SentryAccessDeniedException;
 import org.apache.sentry.provider.db.SentryAlreadyExistsException;
 import org.apache.sentry.provider.db.SentryInvalidInputException;
 import org.apache.sentry.provider.db.SentryNoSuchObjectException;
@@ -32,6 +33,7 @@ import org.apache.sentry.provider.db.service.persistent.SentryStore;
 import org.apache.sentry.provider.db.service.thrift.PolicyStoreConstants.PolicyStoreServerConfig;
 import org.apache.sentry.service.thrift.Status;
 import org.apache.sentry.service.thrift.TSentryResponseStatus;
+import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig;
 import org.apache.thrift.TException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -39,6 +41,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Splitter;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
@@ -52,6 +55,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
   private final Configuration conf;
   private final SentryStore sentryStore;
   private final NotificationHandlerInvoker notificationHandlerInvoker;
+  private final ImmutableSet<String> adminGroups;
   private boolean isReady;
 
   public SentryPolicyStoreProcessor(String name, Configuration conf) throws Exception {
@@ -63,6 +67,8 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     isReady = false;
     sentryStore = new SentryStore(conf);
     isReady = true;
+    adminGroups = ImmutableSet.copyOf(toTrimedLower(Sets.newHashSet(conf.getStrings(
+        ServerConfig.ADMIN_GROUPS, new String[]{}))));
   }
 
   public void stop() {
@@ -145,11 +151,31 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     return privilegeName.toString();
   }
 
+  private static Set<String> toTrimedLower(Set<String> s) {
+    Set<String> result = Sets.newHashSet();
+    for (String v : s) {
+      result.add(v.trim().toLowerCase());
+    }
+    return result;
+  }
+
+  private void authorize(String requestorUser, Set<String> requestorGroups)
+  throws SentryAccessDeniedException {
+    requestorGroups = toTrimedLower(requestorGroups);
+    if (Sets.intersection(adminGroups, requestorGroups).isEmpty()) {
+      String msg = "User: " + requestorUser + " is part of " + requestorGroups +
+          " which does not, intersect admin groups " + adminGroups;
+      LOGGER.warn(msg);
+      throw new SentryAccessDeniedException("Access denied to " + requestorUser);
+    }
+  }
+
   @Override
   public TCreateSentryRoleResponse create_sentry_role(
     TCreateSentryRoleRequest request) throws TException {
     TCreateSentryRoleResponse response = new TCreateSentryRoleResponse();
     try {
+      authorize(request.getRequestorUserName(), request.getRequestorGroupNames());
       CommitContext commitContext = sentryStore.createSentryRole(request.getRoleName(),
           request.getRequestorUserName());
       response.setStatus(Status.OK());
@@ -159,6 +185,9 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       String msg = "Role: " + request + " already exists.";
       LOGGER.error(msg, e);
       response.setStatus(Status.AlreadyExists(msg, e));
+    } catch (SentryAccessDeniedException e) {
+      LOGGER.error(e.getMessage(), e);
+      response.setStatus(Status.AccessDenied(e.getMessage(), e));
     } catch (Exception e) {
       String msg = "Unknown error for request: " + request + ", message: " + e.getMessage();
       LOGGER.error(msg, e);
@@ -173,6 +202,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
 
     TAlterSentryRoleGrantPrivilegeResponse response = new TAlterSentryRoleGrantPrivilegeResponse();
     try {
+      authorize(request.getRequestorUserName(), request.getRequestorGroupNames());
       String privilegeName = constructPrivilegeName(request.getPrivilege());
       request.getPrivilege().setPrivilegeName(privilegeName);
       CommitContext commitContext = sentryStore.alterSentryRoleGrantPrivilege(request.getRoleName(),
@@ -188,6 +218,9 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       String msg = "Invalid input privilege object";
       LOGGER.error(msg, e);
       response.setStatus(Status.InvalidInput(msg, e));
+    } catch (SentryAccessDeniedException e) {
+      LOGGER.error(e.getMessage(), e);
+      response.setStatus(Status.AccessDenied(e.getMessage(), e));
     } catch (Exception e) {
       String msg = "Unknown error for request: " + request + ", message: " + e.getMessage();
       LOGGER.error(msg, e);
@@ -202,6 +235,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
   (TAlterSentryRoleRevokePrivilegeRequest request) throws TException {
     TAlterSentryRoleRevokePrivilegeResponse response = new TAlterSentryRoleRevokePrivilegeResponse();
     try {
+      authorize(request.getRequestorUserName(), request.getRequestorGroupNames());
       String privilegeName = constructPrivilegeName(request.getPrivilege());
       request.getPrivilege().setPrivilegeName(privilegeName);
       CommitContext commitContext = sentryStore.alterSentryRoleRevokePrivilege(request.getRoleName(),
@@ -217,6 +251,9 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       String msg = "Invalid input privilege object";
       LOGGER.error(msg, e);
       response.setStatus(Status.InvalidInput(msg, e));
+    } catch (SentryAccessDeniedException e) {
+      LOGGER.error(e.getMessage(), e);
+      response.setStatus(Status.AccessDenied(e.getMessage(), e));
     } catch (Exception e) {
       String msg = "Unknown error for request: " + request + ", message: " + e.getMessage();
       LOGGER.error(msg, e);
@@ -232,6 +269,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     TDropSentryRoleResponse response = new TDropSentryRoleResponse();
     TSentryResponseStatus status;
     try {
+      authorize(request.getRequestorUserName(), request.getRequestorGroupNames());
       CommitContext commitContext = sentryStore.dropSentryRole(request.getRoleName());
       response.setStatus(Status.OK());
       notificationHandlerInvoker.drop_sentry_role(commitContext,
@@ -240,6 +278,9 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       String msg = "Role :" + request + " does not exist.";
       LOGGER.error(msg, e);
       response.setStatus(Status.NoSuchObject(msg, e));
+    } catch (SentryAccessDeniedException e) {
+      LOGGER.error(e.getMessage(), e);
+      response.setStatus(Status.AccessDenied(e.getMessage(), e));
     } catch (Exception e) {
       String msg = "Unknown error for request: " + request + ", message: " + e.getMessage();
       LOGGER.error(msg, e);
@@ -253,6 +294,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     TAlterSentryRoleAddGroupsRequest request) throws TException {
     TAlterSentryRoleAddGroupsResponse response = new TAlterSentryRoleAddGroupsResponse();
     try {
+      authorize(request.getRequestorUserName(), request.getRequestorGroupNames());
       CommitContext commitContext = sentryStore.alterSentryRoleAddGroups(request.getRequestorUserName(),
                                     request.getRoleName(), request.getGroups());
       response.setStatus(Status.OK());
@@ -262,6 +304,9 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       String msg = "Role: " + request + " does not exist.";
       LOGGER.error(msg, e);
       response.setStatus(Status.NoSuchObject(msg, e));
+    } catch (SentryAccessDeniedException e) {
+      LOGGER.error(e.getMessage(), e);
+      response.setStatus(Status.AccessDenied(e.getMessage(), e));
     } catch (Exception e) {
       String msg = "Unknown error for request: " + request + ", message: " + e.getMessage();
       LOGGER.error(msg, e);
@@ -275,6 +320,7 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
     TAlterSentryRoleDeleteGroupsRequest request) throws TException {
     TAlterSentryRoleDeleteGroupsResponse response = new TAlterSentryRoleDeleteGroupsResponse();
     try {
+      authorize(request.getRequestorUserName(), request.getRequestorGroupNames());
       CommitContext commitContext = sentryStore.alterSentryRoleDeleteGroups(request.getRoleName(),
           request.getGroups());
       response.setStatus(Status.OK());
@@ -284,6 +330,9 @@ public class SentryPolicyStoreProcessor implements SentryPolicyService.Iface {
       String msg = "Role: " + request + " does not exist.";
       LOGGER.error(msg, e);
       response.setStatus(Status.NoSuchObject(msg, e));
+    } catch (SentryAccessDeniedException e) {
+      LOGGER.error(e.getMessage(), e);
+      response.setStatus(Status.AccessDenied(e.getMessage(), e));
     } catch (Exception e) {
       String msg = "Unknown error adding groups to role: " + request;
       LOGGER.error(msg, e);