You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sentry.apache.org by pr...@apache.org on 2014/03/27 17:41:57 UTC

git commit: SENTRY-149: Support SET ROLE (Brock Noland via Prasad Mujumdar)

Repository: incubator-sentry
Updated Branches:
  refs/heads/master 369e241fb -> a6821a611


SENTRY-149: Support SET ROLE (Brock Noland via Prasad Mujumdar)


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

Branch: refs/heads/master
Commit: a6821a611c0a9a99ad92b4bee362c1402264c9f7
Parents: 369e241
Author: Prasad Mujumdar <pr...@cloudera.com>
Authored: Thu Mar 27 09:41:32 2014 -0700
Committer: Prasad Mujumdar <pr...@cloudera.com>
Committed: Thu Mar 27 09:41:32 2014 -0700

----------------------------------------------------------------------
 pom.xml                                         |  8 ++--
 .../hive/ql/exec/SentryGrantRevokeTask.java     | 23 ++++++++++--
 .../binding/hive/HiveAuthzBindingHook.java      |  1 +
 .../SentryHiveAuthorizationTaskFactoryImpl.java | 31 ++++++++++++++--
 .../binding/hive/authz/HiveAuthzBinding.java    | 39 ++++++++++++++++++--
 .../sentry/binding/hive/conf/HiveAuthzConf.java |  5 +++
 .../TestSentryHiveAuthorizationTaskFactory.java | 20 +++++-----
 .../sentry/core/model/db/AccessConstants.java   |  6 +++
 .../tests/e2e/hive/TestDatabaseProvider.java    | 11 +++++-
 9 files changed, 119 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a6821a61/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5bda553..b053bbf 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-SNAPSHOT</hive.version>
-    <hadoop.version>2.2.0-cdh5.0.0-SNAPSHOT</hadoop.version>
+    <hive.version>0.12.0-cdh5.1.0-SNAPSHOT</hive.version>
+    <hadoop.version>2.3.0-cdh5.1.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-SNAPSHOT</solr.sentry.handlers.version>
-    <zookeeper.version>3.4.5-cdh5.0.0-SNAPSHOT</zookeeper.version>
+    <solr.sentry.handlers.version>4.4.0-cdh5.1.0-SNAPSHOT</solr.sentry.handlers.version>
+    <zookeeper.version>3.4.5-cdh5.1.0-SNAPSHOT</zookeeper.version>
   </properties>
 
   <dependencyManagement>

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a6821a61/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
----------------------------------------------------------------------
diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
index 70b05b6..2776eae 100644
--- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
+++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java
@@ -21,6 +21,7 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
@@ -45,9 +46,12 @@ 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.authz.HiveAuthzBinding;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf;
 import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
+import org.apache.sentry.core.common.ActiveRoleSet;
 import org.apache.sentry.core.common.Subject;
+import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient;
 import org.apache.sentry.service.thrift.SentryServiceClientFactory;
 import org.slf4j.Logger;
@@ -57,6 +61,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Splitter;
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
 
 // TODO remove this suppress
 @SuppressWarnings("unused")
@@ -73,6 +78,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
   private SentryServiceClientFactory sentryClientFactory;
   private SentryPolicyServiceClient sentryClient;
   private HiveConf conf;
+  private HiveAuthzBinding hiveAuthzBinding;
   private HiveAuthzConf authzConf;
   private String server;
   private Subject subject;
@@ -104,6 +110,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
         LOG.error(msg, e);
         throw new RuntimeException(msg, e);
       }
+      Preconditions.checkNotNull(hiveAuthzBinding, "HiveAuthzBinding cannot be null");
       Preconditions.checkNotNull(authzConf, "HiveAuthConf cannot be null");
       Preconditions.checkNotNull(subject, "Subject cannot be null");
       Preconditions.checkNotNull(subjectGroups, "Subject Groups cannot be null");
@@ -111,7 +118,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
           "Config " + AuthzConfVars.AUTHZ_SERVER_NAME.getVar() + " is required");
       if (work.getRoleDDLDesc() != null) {
         return processRoleDDL(conf, console, sentryClient, subject.getName(), subjectGroups,
-            work.getRoleDDLDesc());
+            hiveAuthzBinding, work.getRoleDDLDesc());
       }
       if (work.getGrantDesc() != null) {
         return processGrantDDL(conf, console, sentryClient, subject.getName(), subjectGroups,
@@ -148,6 +155,11 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
         "setAuthzConf should only be called once: " + this.authzConf);
     this.authzConf = authzConf;
   }
+  public void setHiveAuthzBinding(HiveAuthzBinding hiveAuthzBinding) {
+    Preconditions.checkState(this.hiveAuthzBinding == null,
+        "setHiveAuthzBinding should only be called once: " + this.hiveAuthzBinding);
+    this.hiveAuthzBinding = hiveAuthzBinding;
+  }
   public void setSubject(Subject subject) {
     Preconditions.checkState(this.subject == null,
         "setSubject should only be called once: " + this.subject);
@@ -162,13 +174,16 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable
   @VisibleForTesting
   static int processRoleDDL(HiveConf conf, LogHelper console,
       SentryPolicyServiceClient sentryClient, String subject,
-      Set<String> subjectGroups, RoleDDLDesc desc) throws SentryUserException {
+      Set<String> subjectGroups, HiveAuthzBinding hiveAuthzBinding, 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();
+      if (operation.equals(RoleDDLDesc.RoleOperation.SET_ROLE)) {
+        hiveAuthzBinding.setActiveRoleSet(name);
+        return RETURN_CODE_SUCCESS;
+      } else if (operation.equals(RoleDDLDesc.RoleOperation.CREATE_ROLE)) {
         sentryClient.createRole(subject, subjectGroups, name);
         return RETURN_CODE_SUCCESS;
       } else if (operation.equals(RoleDDLDesc.RoleOperation.DROP_ROLE)) {

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a6821a61/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 0b83299..8e5cf4e 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
@@ -277,6 +277,7 @@ implements HiveDriverFilterHook {
     for (Task<? extends Serializable> task : rootTasks) {
       if (task instanceof SentryGrantRevokeTask) {
         SentryGrantRevokeTask sentryTask = (SentryGrantRevokeTask)task;
+        sentryTask.setHiveAuthzBinding(hiveAuthzBinding);
         sentryTask.setAuthzConf(authzConf);
         sentryTask.setSubject(subject);
         sentryTask.setSubjectGroups(subjectGroups);

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a6821a61/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
index 252d93b..d735e1b 100644
--- 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
@@ -48,6 +48,9 @@ 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;
+import org.apache.sentry.core.model.db.AccessConstants;
+
+import com.google.common.base.Preconditions;
 
 public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorizationTaskFactory {
 
@@ -58,15 +61,23 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
 
   @Override
   public Task<? extends Serializable> createCreateRoleTask(ASTNode ast, HashSet<ReadEntity> inputs,
-      HashSet<WriteEntity> outputs) {
+      HashSet<WriteEntity> outputs) throws SemanticException {
     String roleName = BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText());
+    if (AccessConstants.RESERVED_ROLE_NAMES.contains(roleName.toUpperCase())) {
+      String msg = "Roles cannot be one of the reserved roles: " + AccessConstants.RESERVED_ROLE_NAMES;
+      throw new SemanticException(msg);
+    }
     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) {
+      HashSet<WriteEntity> outputs) throws SemanticException {
     String roleName = BaseSemanticAnalyzer.unescapeIdentifier(ast.getChild(0).getText());
+    if (AccessConstants.RESERVED_ROLE_NAMES.contains(roleName.toUpperCase())) {
+      String msg = "Roles cannot be one of the reserved roles: " + AccessConstants.RESERVED_ROLE_NAMES;
+      throw new SemanticException(msg);
+    }
     RoleDDLDesc roleDesc = new RoleDDLDesc(roleName, RoleDDLDesc.RoleOperation.DROP_ROLE);
     return createTask(new DDLWork(inputs, outputs, roleDesc));
   }
@@ -121,6 +132,7 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
         && SessionState.get().getAuthenticator() != null) {
       userName = SessionState.get().getAuthenticator().getUserName();
     }
+    Preconditions.checkNotNull(privilegeObj, "privilegeObj is null for " + ast.dump());
     if (privilegeObj.getPartSpec() != null) {
       throw new SemanticException(SentryHiveConstants.PARTITION_PRIVS_NOT_SUPPORTED);
     }
@@ -258,6 +270,19 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
     return createTask(new DDLWork(inputs, outputs, grantRevokeRoleDDL));
   }
 
+  @Override
+  public Task<? extends Serializable> createSetRoleTask(String role, HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs) {
+    RoleDDLDesc roleDesc = new RoleDDLDesc(role, RoleDDLDesc.RoleOperation.SET_ROLE);
+    return createTask(new DDLWork(inputs, outputs, roleDesc));
+  }
+
+  @Override
+  public Task<? extends Serializable> createShowCurrentRoleTask(HashSet<ReadEntity> inputs,
+      HashSet<WriteEntity> outputs, Path resultFile) throws SemanticException {
+    throw new SemanticException("TODO IN FOLLOW ON");
+  }
+
   private PrivilegeObjectDesc analyzePrivilegeObject(ASTNode ast)
       throws SemanticException {
     PrivilegeObjectDesc subject = new PrivilegeObjectDesc();
@@ -327,4 +352,4 @@ public class SentryHiveAuthorizationTaskFactoryImpl implements HiveAuthorization
     task.setWork(work);
     return task;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a6821a61/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 eddf3ae..7a561ef 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
@@ -18,6 +18,7 @@ package org.apache.sentry.binding.hive.authz;
 
 import java.lang.reflect.Constructor;
 import java.util.EnumSet;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -36,6 +37,7 @@ import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars;
 import org.apache.sentry.binding.hive.conf.InvalidConfigurationException;
 import org.apache.sentry.core.common.ActiveRoleSet;
 import org.apache.sentry.core.common.Subject;
+import org.apache.sentry.core.model.db.AccessConstants;
 import org.apache.sentry.core.model.db.DBModelAction;
 import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.apache.sentry.core.model.db.DBModelAuthorizable.AuthorizableType;
@@ -47,7 +49,9 @@ import org.apache.sentry.provider.common.ProviderBackend;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Splitter;
 import com.google.common.base.Strings;
+import com.google.common.collect.Sets;
 
 public class HiveAuthzBinding {
   private static final Logger LOG = LoggerFactory
@@ -55,16 +59,39 @@ public class HiveAuthzBinding {
   private static final Map<String, HiveAuthzBinding> authzBindingMap =
       new ConcurrentHashMap<String, HiveAuthzBinding>();
   private static final AtomicInteger queryID = new AtomicInteger();
+  private static final Splitter ROLE_SET_SPLITTER = Splitter.on(",").trimResults()
+      .omitEmptyStrings();
   public static final String HIVE_BINDING_TAG = "hive.authz.bindings.tag";
 
+  private final HiveConf hiveConf;
   private final Server authServer;
   private final AuthorizationProvider authProvider;
   private volatile boolean open;
+  private ActiveRoleSet activeRoleSet;
 
   public HiveAuthzBinding (HiveConf hiveConf, HiveAuthzConf authzConf) throws Exception {
+    this.hiveConf = hiveConf;
     this.authServer = new Server(authzConf.get(AuthzConfVars.AUTHZ_SERVER_NAME.getVar()));
     this.authProvider = getAuthProvider(hiveConf, authzConf, authServer.getName());
     this.open = true;
+    this.activeRoleSet = parseActiveRoleSet(hiveConf.get(HiveAuthzConf.SENTRY_ACTIVE_ROLE_SET,
+        authzConf.get(HiveAuthzConf.SENTRY_ACTIVE_ROLE_SET, "")).trim());
+  }
+
+  private static ActiveRoleSet parseActiveRoleSet(String name) {
+    // if unset, then we choose the default of ALL
+    if (name.isEmpty()) {
+      return ActiveRoleSet.ALL;
+    } else if (AccessConstants.NONE_ROLE.equalsIgnoreCase(name)) {
+      return new ActiveRoleSet(new HashSet<String>());
+    } else if (AccessConstants.ALL_ROLE.equalsIgnoreCase(name)) {
+      return ActiveRoleSet.ALL;
+    } else if (AccessConstants.RESERVED_ROLE_NAMES.contains(name.toUpperCase())) {
+      String msg = "Role " + name + " is reserved";
+      throw new IllegalArgumentException(msg);
+    } else {
+      return new ActiveRoleSet(Sets.newHashSet(ROLE_SET_SPLITTER.split(name)));
+    }
   }
 
   /**
@@ -179,7 +206,8 @@ public class HiveAuthzBinding {
    * @throws AuthorizationException
    */
   public void authorize(HiveOperation hiveOp, HiveAuthzPrivileges stmtAuthPrivileges,
-      Subject subject, List<List<DBModelAuthorizable>> inputHierarchyList, List<List<DBModelAuthorizable>> outputHierarchyList )
+      Subject subject, List<List<DBModelAuthorizable>> inputHierarchyList,
+      List<List<DBModelAuthorizable>> outputHierarchyList)
           throws AuthorizationException {
     if (!open) {
       throw new IllegalStateException("Binding has been closed");
@@ -210,7 +238,7 @@ public class HiveAuthzBinding {
         if (requiredInputPrivileges.containsKey(getAuthzType(inputHierarchy))) {
           EnumSet<DBModelAction> inputPrivSet =
             requiredInputPrivileges.get(getAuthzType(inputHierarchy));
-          if (!authProvider.hasAccess(subject, inputHierarchy, inputPrivSet, ActiveRoleSet.ALL)) {
+          if (!authProvider.hasAccess(subject, inputHierarchy, inputPrivSet, activeRoleSet)) {
             throw new AuthorizationException("User " + subject.getName() +
                 " does not have privileges for " + hiveOp.name());
           }
@@ -228,7 +256,7 @@ public class HiveAuthzBinding {
         if (requiredOutputPrivileges.containsKey(getAuthzType(outputHierarchy))) {
           EnumSet<DBModelAction> outputPrivSet =
             requiredOutputPrivileges.get(getAuthzType(outputHierarchy));
-          if (!authProvider.hasAccess(subject, outputHierarchy, outputPrivSet, ActiveRoleSet.ALL)) {
+          if (!authProvider.hasAccess(subject, outputHierarchy, outputPrivSet, activeRoleSet)) {
             throw new AuthorizationException("User " + subject.getName() +
                 " does not have priviliedges for " + hiveOp.name());
           }
@@ -236,6 +264,11 @@ public class HiveAuthzBinding {
       }
   }
 
+  public void setActiveRoleSet(String activeRoleSet) {
+    this.activeRoleSet = parseActiveRoleSet(activeRoleSet);
+    hiveConf.set(HiveAuthzConf.SENTRY_ACTIVE_ROLE_SET, activeRoleSet);
+  }
+
   public Set<String> getGroups(Subject subject) {
     return authProvider.getGroupMapping().getGroups(subject.getName());
   }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a6821a61/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 336b925..e162bbd 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
@@ -42,6 +42,11 @@ public class HiveAuthzConf extends Configuration {
   public static final String HIVE_SENTRY_MOCK_COMPILATION = "hive.sentry.mock.compilation";
   public static final String HIVE_SENTRY_MOCK_ERROR = "hive.sentry.mock.error";
   public static final String HIVE_SENTRY_PRIVILEGE_ERROR_MESSAGE = "No valid privileges";
+  /**
+   * Property used to persist the role set in the session. This is not public for now.
+   */
+  public static final String SENTRY_ACTIVE_ROLE_SET = "hive.sentry.active.role.set";
+
 
   /**
    * Config setting definitions

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a6821a61/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
index 817537d..a61f609 100644
--- 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
@@ -55,7 +55,7 @@ import org.mockito.Mockito;
 
 public class TestSentryHiveAuthorizationTaskFactory {
 
-  private static final String SELECT = "SELECT";
+  private static final String ALL = "ALL";
   private static final String DB = "default";
   private static final String TABLE = "table1";
   private static final String GROUP = "group1";
@@ -124,7 +124,7 @@ public class TestSentryHiveAuthorizationTaskFactory {
    */
   @Test
   public void testGrantUserTable() throws Exception {
-    expectSemanticException("GRANT " + SELECT + " ON TABLE " + TABLE + " TO USER " + USER,
+    expectSemanticException("GRANT " + ALL + " ON TABLE " + TABLE + " TO USER " + USER,
         SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "USER");
   }
 
@@ -133,7 +133,7 @@ public class TestSentryHiveAuthorizationTaskFactory {
    */
   @Test
   public void testGrantRoleTable() throws Exception {
-    DDLWork work = analyze(parse("GRANT " + SELECT + " ON TABLE " + TABLE
+    DDLWork work = analyze(parse("GRANT " + ALL + " ON TABLE " + TABLE
         + " TO ROLE " + ROLE));
     GrantDesc grantDesc = work.getGrantDesc();
     Assert.assertNotNull("Grant should not be null", grantDesc);
@@ -142,7 +142,7 @@ public class TestSentryHiveAuthorizationTaskFactory {
       Assert.assertEquals(ROLE, principal.getName());
     }
     for (PrivilegeDesc privilege : assertSize(1, grantDesc.getPrivileges())) {
-      Assert.assertEquals(Privilege.SELECT, privilege.getPrivilege());
+      Assert.assertEquals(Privilege.ALL, privilege.getPrivilege());
     }
     Assert.assertTrue("Expected table", grantDesc.getPrivilegeSubjectDesc()
         .getTable());
@@ -153,7 +153,7 @@ public class TestSentryHiveAuthorizationTaskFactory {
    */
   @Test
   public void testGrantRoleTableWithGrantOption() throws Exception {
-    expectSemanticException("GRANT " + SELECT + " ON TABLE " + TABLE + " TO ROLE " + ROLE +
+    expectSemanticException("GRANT " + ALL + " ON TABLE " + TABLE + " TO ROLE " + ROLE +
         " WITH GRANT OPTION", "Sentry does not allow WITH GRANT OPTION");
   }
 
@@ -162,7 +162,7 @@ public class TestSentryHiveAuthorizationTaskFactory {
    */
   @Test
   public void testGrantGroupTable() throws Exception {
-    expectSemanticException("GRANT " + SELECT + " ON TABLE " + TABLE + " TO GROUP " + GROUP,
+    expectSemanticException("GRANT " + ALL + " ON TABLE " + TABLE + " TO GROUP " + GROUP,
         SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "GROUP");
   }
 
@@ -171,7 +171,7 @@ public class TestSentryHiveAuthorizationTaskFactory {
    */
   @Test
   public void testRevokeUserTable() throws Exception {
-    expectSemanticException("REVOKE " + SELECT + " ON TABLE " + TABLE + " FROM USER " + USER,
+    expectSemanticException("REVOKE " + ALL + " ON TABLE " + TABLE + " FROM USER " + USER,
         SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "USER");
   }
 
@@ -180,7 +180,7 @@ public class TestSentryHiveAuthorizationTaskFactory {
    */
   @Test
   public void testRevokeRoleTable() throws Exception {
-    DDLWork work = analyze(parse("REVOKE " + SELECT + " ON TABLE " + TABLE
+    DDLWork work = analyze(parse("REVOKE " + ALL + " ON TABLE " + TABLE
         + " FROM ROLE " + ROLE));
     RevokeDesc grantDesc = work.getRevokeDesc();
     Assert.assertNotNull("Revoke should not be null", grantDesc);
@@ -189,7 +189,7 @@ public class TestSentryHiveAuthorizationTaskFactory {
       Assert.assertEquals(ROLE, principal.getName());
     }
     for (PrivilegeDesc privilege : assertSize(1, grantDesc.getPrivileges())) {
-      Assert.assertEquals(Privilege.SELECT, privilege.getPrivilege());
+      Assert.assertEquals(Privilege.ALL, privilege.getPrivilege());
     }
     Assert.assertTrue("Expected table", grantDesc.getPrivilegeSubjectDesc()
         .getTable());
@@ -201,7 +201,7 @@ public class TestSentryHiveAuthorizationTaskFactory {
    */
   @Test
   public void testRevokeGroupTable() throws Exception {
-    expectSemanticException("REVOKE " + SELECT + " ON TABLE " + TABLE + " FROM GROUP " + GROUP,
+    expectSemanticException("REVOKE " + ALL + " ON TABLE " + TABLE + " FROM GROUP " + GROUP,
         SentryHiveConstants.GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL + "GROUP");
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a6821a61/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/AccessConstants.java
----------------------------------------------------------------------
diff --git a/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/AccessConstants.java b/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/AccessConstants.java
index 4be391f..9f5035e 100644
--- a/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/AccessConstants.java
+++ b/sentry-core/sentry-core-model-db/src/main/java/org/apache/sentry/core/model/db/AccessConstants.java
@@ -16,6 +16,8 @@
  */
 package org.apache.sentry.core.model.db;
 
+import com.google.common.collect.ImmutableSet;
+
 public class AccessConstants {
 
   /**
@@ -27,4 +29,8 @@ public class AccessConstants {
   public static final String SELECT = "select";
   public static final String INSERT = "insert";
 
+  public static final String ALL_ROLE = "ALL", DEFAULT_ROLE = "DEFAULT", NONE_ROLE = "NONE",
+      SUPERUSER_ROLE = "SUPERUSER", PUBLIC_ROLE = "PUBLIC";
+  public static final ImmutableSet<String> RESERVED_ROLE_NAMES = ImmutableSet.of(ALL_ROLE,
+      DEFAULT_ROLE, NONE_ROLE, SUPERUSER_ROLE, PUBLIC_ROLE);
 }

http://git-wip-us.apache.org/repos/asf/incubator-sentry/blob/a6821a61/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestDatabaseProvider.java
----------------------------------------------------------------------
diff --git a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestDatabaseProvider.java b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestDatabaseProvider.java
index b8163b3..0e72a8b 100644
--- a/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestDatabaseProvider.java
+++ b/sentry-tests/sentry-tests-hive/src/test/java/org/apache/sentry/tests/e2e/hive/TestDatabaseProvider.java
@@ -116,9 +116,18 @@ public class TestDatabaseProvider extends AbstractTestWithHiveServer {
     connection = context.createConnection(USER1_1);
     statement = context.createStatement(connection);
     context.assertSentryServiceAccessDenied(statement, "CREATE ROLE r2");
+    // test default of ALL
+    statement.execute("SELECT * FROM t1");
+    // test a specific role
+    statement.execute("SET ROLE user_role");
+    statement.execute("SELECT * FROM t1");
+    // test NONE
+    statement.execute("SET ROLE NONE");
+    context.assertAuthzException(statement, "SELECT * FROM t1");
+    // test ALL
+    statement.execute("SET ROLE ALL");
     statement.execute("SELECT * FROM t1");
     statement.close();
     connection.close();
-
   }
 }