You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2018/08/16 00:29:47 UTC

[3/6] impala git commit: IMPALA-7342: Add initial support for user-level permissions

http://git-wip-us.apache.org/repos/asf/impala/blob/a23e6f29/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
index 40f604d..cdf85ed 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -51,7 +51,6 @@ import org.apache.hadoop.hive.metastore.api.SerDeInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.StringColumnStatsData;
 import org.apache.impala.analysis.AlterTableSortByStmt;
-import org.apache.impala.analysis.ColumnName;
 import org.apache.impala.analysis.FunctionName;
 import org.apache.impala.analysis.TableName;
 import org.apache.impala.authorization.User;
@@ -75,8 +74,8 @@ import org.apache.impala.catalog.KuduTable;
 import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
 import org.apache.impala.catalog.PartitionNotFoundException;
 import org.apache.impala.catalog.PartitionStatsUtil;
+import org.apache.impala.catalog.PrincipalPrivilege;
 import org.apache.impala.catalog.Role;
-import org.apache.impala.catalog.RolePrivilege;
 import org.apache.impala.catalog.RowFormat;
 import org.apache.impala.catalog.ScalarFunction;
 import org.apache.impala.catalog.Table;
@@ -2989,7 +2988,7 @@ public class CatalogOpExecutor {
 
     TCatalogObject catalogObject = new TCatalogObject();
     catalogObject.setType(role.getCatalogObjectType());
-    catalogObject.setRole(role.toThrift());
+    catalogObject.setPrincipal(role.toThrift());
     catalogObject.setCatalog_version(role.getCatalogVersion());
     if (createDropRoleParams.isIs_drop()) {
       resp.result.addToRemoved_catalog_objects(catalogObject);
@@ -3024,7 +3023,7 @@ public class CatalogOpExecutor {
     Preconditions.checkNotNull(role);
     TCatalogObject catalogObject = new TCatalogObject();
     catalogObject.setType(role.getCatalogObjectType());
-    catalogObject.setRole(role.toThrift());
+    catalogObject.setPrincipal(role.toThrift());
     catalogObject.setCatalog_version(role.getCatalogVersion());
     resp.result.addToUpdated_catalog_objects(catalogObject);
     if (grantRevokeRoleParams.isIs_grant()) {
@@ -3046,7 +3045,7 @@ public class CatalogOpExecutor {
     verifySentryServiceEnabled();
     String roleName = grantRevokePrivParams.getRole_name();
     List<TPrivilege> privileges = grantRevokePrivParams.getPrivileges();
-    List<RolePrivilege> rolePrivileges = null;
+    List<PrincipalPrivilege> rolePrivileges = null;
     if (grantRevokePrivParams.isIs_grant()) {
       rolePrivileges = catalog_.getSentryProxy().grantRolePrivileges(requestingUser,
           roleName, privileges);
@@ -3058,7 +3057,7 @@ public class CatalogOpExecutor {
     }
     Preconditions.checkNotNull(rolePrivileges);
     List<TCatalogObject> updatedPrivs = Lists.newArrayList();
-    for (RolePrivilege rolePriv: rolePrivileges) {
+    for (PrincipalPrivilege rolePriv: rolePrivileges) {
       updatedPrivs.add(rolePriv.toTCatalogObject());
     }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/a23e6f29/fe/src/main/java/org/apache/impala/service/JniFrontend.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/JniFrontend.java b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
index f02db61..ce16b51 100644
--- a/fe/src/main/java/org/apache/impala/service/JniFrontend.java
+++ b/fe/src/main/java/org/apache/impala/service/JniFrontend.java
@@ -45,7 +45,6 @@ import org.apache.impala.analysis.ToSqlUtils;
 import org.apache.impala.authorization.AuthorizationConfig;
 import org.apache.impala.authorization.ImpalaInternalAdminUser;
 import org.apache.impala.authorization.User;
-import org.apache.impala.catalog.DataSource;
 import org.apache.impala.catalog.FeDataSource;
 import org.apache.impala.catalog.FeDb;
 import org.apache.impala.catalog.Function;

http://git-wip-us.apache.org/repos/asf/impala/blob/a23e6f29/fe/src/main/java/org/apache/impala/util/SentryPolicyService.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/util/SentryPolicyService.java b/fe/src/main/java/org/apache/impala/util/SentryPolicyService.java
index 8fc72c9..f039ee7 100644
--- a/fe/src/main/java/org/apache/impala/util/SentryPolicyService.java
+++ b/fe/src/main/java/org/apache/impala/util/SentryPolicyService.java
@@ -19,6 +19,7 @@ package org.apache.impala.util;
 
 import java.util.List;
 
+import org.apache.impala.catalog.PrincipalPrivilege;
 import org.apache.sentry.api.service.thrift.SentryPolicyServiceClient;
 import org.apache.sentry.api.service.thrift.TSentryGrantOption;
 import org.apache.sentry.api.service.thrift.TSentryPrivilege;
@@ -31,7 +32,6 @@ import org.apache.impala.analysis.PrivilegeSpec;
 import org.apache.impala.authorization.SentryConfig;
 import org.apache.impala.authorization.User;
 import org.apache.impala.catalog.AuthorizationException;
-import org.apache.impala.catalog.RolePrivilege;
 import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.InternalException;
 import org.apache.impala.thrift.TPrivilege;
@@ -236,7 +236,7 @@ public class SentryPolicyService {
    *
    * @param requestingUser - The requesting user.
    * @param roleName - The role to grant privileges to (case insensitive).
-   * @param privilege - The privilege to grant.
+   * @param privileges - The privileges to grant.
    * @throws ImpalaException - On any error
    */
   public void grantRolePrivileges(User requestingUser, String roleName,
@@ -307,7 +307,7 @@ public class SentryPolicyService {
    *
    * @param requestingUser - The requesting user.
    * @param roleName - The role to revoke privileges from (case insensitive).
-   * @param privilege - The privilege to revoke.
+   * @param privileges - The privileges to revoke.
    * @throws ImpalaException - On any error
    */
   public void revokeRolePrivileges(User requestingUser, String roleName,
@@ -448,7 +448,7 @@ public class SentryPolicyService {
       privilege.setPrivilege_level(Enum.valueOf(TPrivilegeLevel.class,
           sentryPriv.getAction().toUpperCase()));
     }
-    privilege.setPrivilege_name(RolePrivilege.buildRolePrivilegeName(privilege));
+    privilege.setPrivilege_name(PrincipalPrivilege.buildPrivilegeName(privilege));
     privilege.setCreate_time_ms(sentryPriv.getCreateTime());
     if (sentryPriv.isSetGrantOption() &&
         sentryPriv.getGrantOption() == TSentryGrantOption.TRUE) {

http://git-wip-us.apache.org/repos/asf/impala/blob/a23e6f29/fe/src/main/java/org/apache/impala/util/SentryProxy.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/util/SentryProxy.java b/fe/src/main/java/org/apache/impala/util/SentryProxy.java
index 7863923..fa769b6 100644
--- a/fe/src/main/java/org/apache/impala/util/SentryProxy.java
+++ b/fe/src/main/java/org/apache/impala/util/SentryProxy.java
@@ -18,11 +18,18 @@
 package org.apache.impala.util;
 
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.impala.catalog.AuthorizationException;
+import org.apache.impala.catalog.CatalogException;
+import org.apache.impala.catalog.CatalogServiceCatalog;
+import org.apache.impala.catalog.PrincipalPrivilege;
+import org.apache.impala.catalog.Role;
+import org.apache.impala.thrift.TPrincipalType;
 import org.apache.log4j.Logger;
 import org.apache.sentry.api.service.thrift.TSentryGroup;
 import org.apache.sentry.api.service.thrift.TSentryPrivilege;
@@ -30,11 +37,6 @@ import org.apache.sentry.api.service.thrift.TSentryRole;
 
 import org.apache.impala.authorization.SentryConfig;
 import org.apache.impala.authorization.User;
-import org.apache.impala.catalog.AuthorizationException;
-import org.apache.impala.catalog.CatalogException;
-import org.apache.impala.catalog.CatalogServiceCatalog;
-import org.apache.impala.catalog.Role;
-import org.apache.impala.catalog.RolePrivilege;
 import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.ImpalaRuntimeException;
 import org.apache.impala.service.BackendConfig;
@@ -97,10 +99,10 @@ public class SentryProxy {
    * There is currently no way to get a snapshot of the policy from the Sentry Service,
    * so it is possible that Impala will end up in a state that is not consistent with a
    * state the Sentry Service has ever been in. For example, consider the case where a
-   * refresh is running and all privileges for Role A have been processed. Before moving
-   * to Role B, the user revokes a privilege from Role A and grants it to Role B.
-   * Impala will temporarily (until the next refresh) think the privilege is granted to
-   * Role A AND to Role B.
+   * refresh is running and all privileges for Principal A have been processed. Before
+   * moving to Principal B, the user revokes a privilege from Principal A and grants it to
+   * Principal B. Impala will temporarily (until the next refresh) think the privilege is
+   * granted to Principal A AND to Principal B.
    * TODO: Think more about consistency as well as how to recover from errors that leave
    * the policy in a potentially inconsistent state (an RPC fails part-way through a
    * refresh). We should also consider applying this entire update to the catalog
@@ -154,23 +156,25 @@ public class SentryProxy {
                 sentryPolicyService_.listRolePrivileges(processUser_, role.getName());
             } catch (ImpalaException e) {
               String roleName = role.getName() != null ? role.getName(): "null";
-              LOG.error("Error listing the Role name: " + roleName, e);
+              LOG.error("Error listing the role name: " + roleName, e);
             }
 
             // Check all the privileges that are part of this role.
             for (TSentryPrivilege sentryPriv: sentryPrivlist) {
               TPrivilege thriftPriv =
                   SentryPolicyService.sentryPrivilegeToTPrivilege(sentryPriv);
-              thriftPriv.setRole_id(role.getId());
+              thriftPriv.setPrincipal_id(role.getId());
+              thriftPriv.setPrincipal_type(TPrincipalType.ROLE);
+
               privilegesToRemove.remove(thriftPriv.getPrivilege_name().toLowerCase());
 
-              RolePrivilege existingPriv =
+              PrincipalPrivilege existingRolePriv =
                   role.getPrivilege(thriftPriv.getPrivilege_name());
               // We already know about this privilege (privileges cannot be modified).
-              if (existingPriv != null &&
-                  existingPriv.getCreateTimeMs() == sentryPriv.getCreateTime()) {
+              if (existingRolePriv != null &&
+                  existingRolePriv.getCreateTimeMs() == sentryPriv.getCreateTime()) {
                 if (resetVersions_) {
-                  existingPriv.setCatalogVersion(
+                  existingRolePriv.setCatalogVersion(
                       catalog_.incrementAndGetCatalogVersion());
                 }
                 continue;
@@ -288,11 +292,11 @@ public class SentryProxy {
    * Throws exception if there was any error updating the Sentry Service or if the Impala
    * catalog does not contain the given role name.
    */
-  public synchronized List<RolePrivilege> grantRolePrivileges(User user,
+  public synchronized List<PrincipalPrivilege> grantRolePrivileges(User user,
       String roleName, List<TPrivilege> privileges) throws ImpalaException {
     sentryPolicyService_.grantRolePrivileges(user, roleName, privileges);
     // Update the catalog
-    List<RolePrivilege> rolePrivileges = Lists.newArrayList();
+    List<PrincipalPrivilege> rolePrivileges = Lists.newArrayList();
     for (TPrivilege privilege: privileges) {
       rolePrivileges.add(catalog_.addRolePrivilege(roleName, privilege));
     }
@@ -306,15 +310,15 @@ public class SentryProxy {
    * updating the Sentry Service or if the Impala catalog does not contain the given role
    * name.
    */
-  public synchronized List<RolePrivilege> revokeRolePrivileges(User user,
+  public synchronized List<PrincipalPrivilege> revokeRolePrivileges(User user,
       String roleName, List<TPrivilege> privileges, boolean hasGrantOption)
       throws ImpalaException {
-    List<RolePrivilege> rolePrivileges = Lists.newArrayList();
+    List<PrincipalPrivilege> rolePrivileges = Lists.newArrayList();
     if (!hasGrantOption) {
       sentryPolicyService_.revokeRolePrivileges(user, roleName, privileges);
       // Update the catalog
       for (TPrivilege privilege: privileges) {
-        RolePrivilege rolePriv = catalog_.removeRolePrivilege(roleName, privilege);
+        PrincipalPrivilege rolePriv = catalog_.removeRolePrivilege(roleName, privilege);
         if (rolePriv == null) continue;
         rolePrivileges.add(rolePriv);
       }
@@ -326,7 +330,8 @@ public class SentryProxy {
       sentryPolicyService_.revokeRolePrivileges(user, roleName, privileges);
       List<TPrivilege> updatedPrivileges = Lists.newArrayList();
       for (TPrivilege privilege: privileges) {
-        RolePrivilege existingPriv = catalog_.getRolePrivilege(roleName, privilege);
+        PrincipalPrivilege existingPriv = catalog_.getPrincipalPrivilege(roleName,
+            privilege);
         if (existingPriv == null) continue;
         TPrivilege updatedPriv = existingPriv.toThrift();
         updatedPriv.setHas_grant_opt(false);

http://git-wip-us.apache.org/repos/asf/impala/blob/a23e6f29/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
index 2741e5d..dd58131 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
@@ -29,8 +29,8 @@ import org.apache.impala.util.EventSequence;
 import org.junit.Test;
 
 public class AnalyzeAuthStmtsTest extends AnalyzerTest {
-  public AnalyzeAuthStmtsTest() throws AnalysisException {
-    catalog_.getAuthPolicy().addRole(
+  public AnalyzeAuthStmtsTest() {
+    catalog_.getAuthPolicy().addPrincipal(
         new Role("myRole", new HashSet<String>()));
   }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/a23e6f29/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java b/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
index 9a38300..abfff91 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AuthorizationStmtTest.java
@@ -25,8 +25,8 @@ import org.apache.impala.authorization.AuthorizationConfig;
 import org.apache.impala.authorization.PrivilegeRequest;
 import org.apache.impala.authorization.User;
 import org.apache.impala.catalog.AuthorizationException;
+import org.apache.impala.catalog.PrincipalPrivilege;
 import org.apache.impala.catalog.Role;
-import org.apache.impala.catalog.RolePrivilege;
 import org.apache.impala.catalog.ScalarFunction;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.FrontendTestBase;
@@ -38,6 +38,7 @@ import org.apache.impala.thrift.TColumnValue;
 import org.apache.impala.thrift.TDescribeOutputStyle;
 import org.apache.impala.thrift.TDescribeResult;
 import org.apache.impala.thrift.TFunctionBinaryType;
+import org.apache.impala.thrift.TPrincipalType;
 import org.apache.impala.thrift.TPrivilege;
 import org.apache.impala.thrift.TPrivilegeLevel;
 import org.apache.impala.thrift.TPrivilegeScope;
@@ -66,7 +67,7 @@ import static org.junit.Assert.fail;
  */
 public class AuthorizationStmtTest extends FrontendTestBase {
   private static final String SENTRY_SERVER = "server1";
-  private final static User USER = new User(System.getProperty("user.name"));
+  private static final User USER = new User(System.getProperty("user.name"));
   private final AnalysisContext analysisContext_;
   private final SentryPolicyService sentryService_;
   private final ImpaladTestCatalog authzCatalog_;
@@ -963,7 +964,16 @@ public class AuthorizationStmtTest extends FrontendTestBase {
     authorize(String.format("show role grant group `%s`", USER.getName())).ok();
 
     // Show grant role should always be allowed.
-    authorize(String.format("show grant role authz_test_role")).ok();
+    try {
+      authzCatalog_.addRole("test_role");
+      authorize("show grant role test_role").ok();
+      authorize("show grant role test_role on server").ok();
+      authorize("show grant role test_role on database functional").ok();
+      authorize("show grant role test_role on table functional.alltypes").ok();
+      authorize("show grant role test_role on uri '/test-warehouse'").ok();
+    } finally {
+      authzCatalog_.removeRole("test_role");
+    }
 
     // Show create table.
     test = authorize("show create table functional.alltypes");
@@ -2299,10 +2309,51 @@ public class AuthorizationStmtTest extends FrontendTestBase {
     return privLevels.toArray(new TPrivilegeLevel[0]);
   }
 
+  private static abstract class WithPrincipal {
+    protected final AuthzTest test_;
+
+    public WithPrincipal(AuthzTest test) { test_ = test; }
+
+    public abstract void create(TPrivilege[]... privileges) throws ImpalaException;
+    public abstract void drop() throws ImpalaException;
+    public abstract String getName();
+  }
+
+  private static class WithUser extends WithPrincipal {
+    public WithUser(AuthzTest test) { super(test); }
+
+    @Override
+    public void create(TPrivilege[]... privileges) throws ImpalaException {
+      test_.createUser(privileges);
+    }
+
+    @Override
+    public void drop() throws ImpalaException { test_.dropUser(); }
+
+    @Override
+    public String getName() { return test_.user_; }
+  }
+
+  private static class WithRole extends WithPrincipal {
+    public WithRole(AuthzTest test) { super(test); }
+
+    @Override
+    public void create(TPrivilege[]... privileges) throws ImpalaException {
+      test_.createRole(privileges);
+    }
+
+    @Override
+    public void drop() throws ImpalaException { test_.dropRole(); }
+
+    @Override
+    public String getName() { return test_.role_; }
+  }
+
   private class AuthzTest {
     private final AnalysisContext context_;
     private final String stmt_;
     private final String role_ = "authz_test_role";
+    private final String user_ = USER.getName();
 
     public AuthzTest(String stmt) {
       this(null, stmt);
@@ -2319,95 +2370,121 @@ public class AuthorizationStmtTest extends FrontendTestBase {
       authzCatalog_.addRoleGrantGroup(role_, USER.getName());
       for (TPrivilege[] privs: privileges) {
         for (TPrivilege privilege: privs) {
-          privilege.setRole_id(role.getId());
+          privilege.setPrincipal_id(role.getId());
+          privilege.setPrincipal_type(TPrincipalType.ROLE);
           authzCatalog_.addRolePrivilege(role_, privilege);
         }
       }
     }
 
+    private void createUser(TPrivilege[]... privileges) throws ImpalaException {
+      org.apache.impala.catalog.User user = authzCatalog_.addUser(user_);
+      for (TPrivilege[] privs: privileges) {
+        for (TPrivilege privilege: privs) {
+          privilege.setPrincipal_id(user.getId());
+          privilege.setPrincipal_type(TPrincipalType.USER);
+          authzCatalog_.addUserPrivilege(user_, privilege);
+        }
+      }
+    }
+
     private void dropRole() throws ImpalaException {
       authzCatalog_.removeRole(role_);
     }
 
+    private void dropUser() throws ImpalaException {
+      authzCatalog_.removeUser(user_);
+    }
+
     /**
-     * This method runs with the specified privileges.
+     * This method runs with the specified privileges for the role and then for the user.
      *
-     * A new temporary role will be created and assigned to the specified privileges
-     * into the new role. The new role will be dropped once this method finishes.
+     * A new temporary role/user will be created and assigned to the specified privileges
+     * into the new role/user. The new role/user will be dropped once this method
+     * finishes.
      */
     public AuthzTest ok(TPrivilege[]... privileges) throws ImpalaException {
-      try {
-        createRole(privileges);
-        if (context_ != null) {
-          authzOk(context_, stmt_);
-        } else {
-          authzOk(stmt_);
+      for (WithPrincipal withPrincipal: new WithPrincipal[]{
+          new WithRole(this), new WithUser(this)}) {
+        try {
+          withPrincipal.create(privileges);
+          if (context_ != null) {
+            authzOk(context_, stmt_, withPrincipal);
+          } else {
+            authzOk(stmt_, withPrincipal);
+          }
+        } finally {
+          withPrincipal.drop();
         }
-      } catch (AuthorizationException ae) {
-        // Because the same test can be called from multiple statements
-        // it is useful to know which statement caused the exception.
-        throw new AuthorizationException(stmt_ + ": " + ae.getMessage(), ae);
-      } finally {
-        dropRole();
       }
       return this;
     }
 
     /**
-     * This method runs with the specified privileges and checks describe output.
+     * This method runs with the specified privileges and checks describe output for the
+     * role and then the user.
      *
-     * A new temporary role will be created and assigned to the specified privileges
-     * into the new role. The new role will be dropped once this method finishes.
+     * A new temporary role/user will be created and assigned to the specified privileges
+     * into the new role/user. The new role/user will be dropped once this method
+     * finishes.
      */
     public AuthzTest okDescribe(TTableName table, TDescribeOutputStyle style,
         String[] requiredStrings, String[] excludedStrings, TPrivilege[]... privileges)
         throws ImpalaException {
-      try {
-        createRole(privileges);
-        if (context_ != null) {
-          authzOk(context_, stmt_);
-        } else {
-          authzOk(stmt_);
-        }
-        List<String> result = resultToStringList(authzFrontend_.describeTable(table,
-            style, USER));
-        if (requiredStrings != null) {
-          for (String str: requiredStrings) {
-            assertTrue(String.format("\"%s\" is not in the describe output.\n" +
-                "Expected : %s\n" +
-                "Actual   : %s", str, Arrays.toString(requiredStrings), result),
-                result.contains(str));
+      for (WithPrincipal withPrincipal: new WithPrincipal[]{
+          new WithRole(this), new WithUser(this)}) {
+        try {
+          withPrincipal.create(privileges);
+          if (context_ != null) {
+            authzOk(context_, stmt_, withPrincipal);
+          } else {
+            authzOk(stmt_, withPrincipal);
           }
-        }
-        if (excludedStrings != null) {
-          for (String str: excludedStrings) {
-            assertTrue(String.format("\"%s\" should not be in the describe output.", str),
-                !result.contains(str));
+          List<String> result = resultToStringList(authzFrontend_.describeTable(table,
+              style, USER));
+          if (requiredStrings != null) {
+            for (String str : requiredStrings) {
+              assertTrue(String.format("\"%s\" is not in the describe output.\n" +
+                  "Expected : %s\n" +
+                  "Actual   : %s", str, Arrays.toString(requiredStrings), result),
+                  result.contains(str));
+            }
           }
+          if (excludedStrings != null) {
+            for (String str : excludedStrings) {
+              assertTrue(String.format(
+                  "\"%s\" should not be in the describe output.", str),
+                  !result.contains(str));
+            }
+          }
+        } finally {
+          withPrincipal.drop();
         }
-      } finally {
-        dropRole();
       }
       return this;
     }
 
     /**
-     * This method runs with the specified privileges.
+     * This method runs with the specified privileges for the user and then the role.
      *
-     * A new temporary role will be created and assigned to the specified privileges
-     * into the new role. The new role will be dropped once this method finishes.
+     * A new temporary role/user will be created and assigned to the specified privileges
+     * into the new role/user. The new role/user will be dropped once this method
+     * finishes.
      */
     public AuthzTest error(String expectedError, TPrivilege[]... privileges)
         throws ImpalaException {
-      try {
-        createRole(privileges);
-        if (context_ != null) {
-          authzError(context_, stmt_, expectedError);
-        } else {
-          authzError(stmt_, expectedError);
+      for (WithPrincipal withPrincipal: new WithPrincipal[]{
+          new WithRole(this), new WithUser(this)}) {
+        try {
+          withPrincipal.create(privileges);
+          if (context_ != null) {
+            authzError(context_, stmt_, expectedError, withPrincipal);
+          } else {
+            authzError(stmt_, expectedError, withPrincipal);
+          }
+        } finally {
+          withPrincipal.drop();
         }
-      } finally {
-        dropRole();
       }
       return this;
     }
@@ -2426,7 +2503,7 @@ public class AuthorizationStmtTest extends FrontendTestBase {
     for (int i = 0; i < levels.length; i++) {
       privileges[i] = new TPrivilege("", levels[i], TPrivilegeScope.SERVER, false);
       privileges[i].setServer_name(SENTRY_SERVER);
-      privileges[i].setPrivilege_name(RolePrivilege.buildRolePrivilegeName(
+      privileges[i].setPrivilege_name(PrincipalPrivilege.buildPrivilegeName(
           privileges[i]));
     }
     return privileges;
@@ -2438,7 +2515,7 @@ public class AuthorizationStmtTest extends FrontendTestBase {
       privileges[i] = new TPrivilege("", levels[i], TPrivilegeScope.DATABASE, false);
       privileges[i].setServer_name(SENTRY_SERVER);
       privileges[i].setDb_name(db);
-      privileges[i].setPrivilege_name(RolePrivilege.buildRolePrivilegeName(
+      privileges[i].setPrivilege_name(PrincipalPrivilege.buildPrivilegeName(
           privileges[i]));
     }
     return privileges;
@@ -2451,7 +2528,7 @@ public class AuthorizationStmtTest extends FrontendTestBase {
       privileges[i].setServer_name(SENTRY_SERVER);
       privileges[i].setDb_name(db);
       privileges[i].setTable_name(table);
-      privileges[i].setPrivilege_name(RolePrivilege.buildRolePrivilegeName(
+      privileges[i].setPrivilege_name(PrincipalPrivilege.buildPrivilegeName(
           privileges[i]));
     }
     return privileges;
@@ -2474,7 +2551,7 @@ public class AuthorizationStmtTest extends FrontendTestBase {
         privileges[idx].setDb_name(db);
         privileges[idx].setTable_name(table);
         privileges[idx].setColumn_name(column);
-        privileges[idx].setPrivilege_name(RolePrivilege.buildRolePrivilegeName(
+        privileges[idx].setPrivilege_name(PrincipalPrivilege.buildPrivilegeName(
             privileges[idx]));
         idx++;
       }
@@ -2488,47 +2565,56 @@ public class AuthorizationStmtTest extends FrontendTestBase {
       privileges[i] = new TPrivilege("", levels[i], TPrivilegeScope.URI, false);
       privileges[i].setServer_name(SENTRY_SERVER);
       privileges[i].setUri(uri);
-      privileges[i].setPrivilege_name(RolePrivilege.buildRolePrivilegeName(
+      privileges[i].setPrivilege_name(PrincipalPrivilege.buildPrivilegeName(
           privileges[i]));
     }
     return privileges;
   }
 
-  private void authzOk(String stmt) throws ImpalaException {
-    authzOk(analysisContext_, stmt);
+  private void authzOk(String stmt, WithPrincipal withPrincipal) throws ImpalaException {
+    authzOk(analysisContext_, stmt, withPrincipal);
   }
 
-  private void authzOk(AnalysisContext context, String stmt) throws ImpalaException {
-    authzOk(authzFrontend_, context, stmt);
+  private void authzOk(AnalysisContext context, String stmt, WithPrincipal withPrincipal)
+      throws ImpalaException {
+    authzOk(authzFrontend_, context, stmt, withPrincipal);
   }
 
-  private void authzOk(Frontend fe, AnalysisContext context, String stmt)
-      throws ImpalaException {
-    parseAndAnalyze(stmt, context, fe);
+  private void authzOk(Frontend fe, AnalysisContext context, String stmt,
+      WithPrincipal withPrincipal) throws ImpalaException {
+    try {
+      parseAndAnalyze(stmt, context, fe);
+    } catch (AuthorizationException e) {
+      // Because the same test can be called from multiple statements
+      // it is useful to know which statement caused the exception.
+      throw new AuthorizationException(String.format(
+          "\nPrincipal: %s\nStatement: %s\nError: %s", withPrincipal.getName(),
+          stmt, e.getMessage(), e));
+    }
   }
 
   /**
    * Verifies that a given statement fails authorization and the expected error
    * string matches.
    */
-  private void authzError(String stmt, String expectedError, Matcher matcher)
-      throws ImpalaException {
-    authzError(analysisContext_, stmt, expectedError, matcher);
+  private void authzError(String stmt, String expectedError, Matcher matcher,
+      WithPrincipal withPrincipal) throws ImpalaException {
+    authzError(analysisContext_, stmt, expectedError, matcher, withPrincipal);
   }
 
-  private void authzError(String stmt, String expectedError)
+  private void authzError(String stmt, String expectedError, WithPrincipal withPrincipal)
       throws ImpalaException {
-    authzError(analysisContext_, stmt, expectedError, startsWith());
+    authzError(analysisContext_, stmt, expectedError, startsWith(), withPrincipal);
   }
 
   private void authzError(AnalysisContext ctx, String stmt, String expectedError,
-      Matcher matcher) throws ImpalaException {
-    authzError(authzFrontend_, ctx, stmt, expectedError, matcher);
+      Matcher matcher, WithPrincipal withPrincipal) throws ImpalaException {
+    authzError(authzFrontend_, ctx, stmt, expectedError, matcher, withPrincipal);
   }
 
-  private void authzError(AnalysisContext ctx, String stmt, String expectedError)
-      throws ImpalaException {
-    authzError(authzFrontend_, ctx, stmt, expectedError, startsWith());
+  private void authzError(AnalysisContext ctx, String stmt, String expectedError,
+      WithPrincipal withPrincipal) throws ImpalaException {
+    authzError(authzFrontend_, ctx, stmt, expectedError, startsWith(), withPrincipal);
   }
 
   private interface Matcher {
@@ -2553,8 +2639,8 @@ public class AuthorizationStmtTest extends FrontendTestBase {
     };
   }
 
-  private void authzError(Frontend fe, AnalysisContext ctx,
-      String stmt, String expectedErrorString, Matcher matcher)
+  private void authzError(Frontend fe, AnalysisContext ctx, String stmt,
+      String expectedErrorString, Matcher matcher, WithPrincipal withPrincipal)
       throws ImpalaException {
     Preconditions.checkNotNull(expectedErrorString);
     try {
@@ -2568,7 +2654,8 @@ public class AuthorizationStmtTest extends FrontendTestBase {
           matcher.match(errorString, expectedErrorString));
       return;
     }
-    fail("Stmt didn't result in authorization error: " + stmt);
+    fail(String.format("Statement did not result in authorization error.\n" +
+        "Principal: %s\nStatement: %s", withPrincipal.getName(), stmt));
   }
 
   private void verifyPrivilegeReqs(String stmt, Set<String> expectedPrivilegeNames)

http://git-wip-us.apache.org/repos/asf/impala/blob/a23e6f29/fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/catalog/CatalogTest.java b/fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
index 7ff5054..ad1595b 100644
--- a/fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
+++ b/fe/src/test/java/org/apache/impala/catalog/CatalogTest.java
@@ -18,14 +18,17 @@
 package org.apache.impala.catalog;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
@@ -44,6 +47,10 @@ import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
 import org.apache.impala.common.Reference;
 import org.apache.impala.testutil.CatalogServiceTestCatalog;
 import org.apache.impala.thrift.TFunctionBinaryType;
+import org.apache.impala.thrift.TPrincipalType;
+import org.apache.impala.thrift.TPrivilege;
+import org.apache.impala.thrift.TPrivilegeLevel;
+import org.apache.impala.thrift.TPrivilegeScope;
 import org.apache.impala.thrift.TTableName;
 import org.junit.Test;
 
@@ -699,4 +706,84 @@ public class CatalogTest {
     fnNames = getFunctionSignatures("default");
     assertEquals(fnNames.size(), 0);
   }
+
+  @Test
+  public void testSentryCatalog() throws CatalogException {
+    AuthorizationPolicy authPolicy = catalog_.getAuthPolicy();
+
+    User user = catalog_.addUser("user1");
+    TPrivilege userPrivilege = new TPrivilege();
+    userPrivilege.setPrincipal_type(TPrincipalType.USER);
+    userPrivilege.setPrincipal_id(user.getId());
+    userPrivilege.setCreate_time_ms(-1);
+    userPrivilege.setServer_name("server1");
+    userPrivilege.setScope(TPrivilegeScope.SERVER);
+    userPrivilege.setPrivilege_level(TPrivilegeLevel.ALL);
+    userPrivilege.setPrivilege_name(PrincipalPrivilege.buildPrivilegeName(userPrivilege));
+    catalog_.addUserPrivilege("user1", userPrivilege);
+    assertSame(user, authPolicy.getPrincipal("user1", TPrincipalType.USER));
+    assertNull(authPolicy.getPrincipal("user2", TPrincipalType.USER));
+    assertNull(authPolicy.getPrincipal("user1", TPrincipalType.ROLE));
+    // Add the same user, the old user will be deleted.
+    user = catalog_.addUser("user1");
+    assertSame(user, authPolicy.getPrincipal("user1", TPrincipalType.USER));
+    // Delete the user.
+    assertSame(user, catalog_.removeUser("user1"));
+    assertNull(authPolicy.getPrincipal("user1", TPrincipalType.USER));
+
+    Role role = catalog_.addRole("role1", Sets.newHashSet("group1", "group2"));
+    TPrivilege rolePrivilege = new TPrivilege();
+    rolePrivilege.setPrincipal_type(TPrincipalType.ROLE);
+    rolePrivilege.setPrincipal_id(role.getId());
+    rolePrivilege.setCreate_time_ms(-1);
+    rolePrivilege.setServer_name("server1");
+    rolePrivilege.setScope(TPrivilegeScope.SERVER);
+    rolePrivilege.setPrivilege_level(TPrivilegeLevel.ALL);
+    rolePrivilege.setPrivilege_name(PrincipalPrivilege.buildPrivilegeName(rolePrivilege));
+    catalog_.addRolePrivilege("role1", rolePrivilege);
+    assertSame(role, catalog_.getAuthPolicy().getPrincipal("role1", TPrincipalType.ROLE));
+    assertNull(catalog_.getAuthPolicy().getPrincipal("role1", TPrincipalType.USER));
+    assertNull(catalog_.getAuthPolicy().getPrincipal("role2", TPrincipalType.ROLE));
+    // Add the same role, the old role will be deleted.
+    role = catalog_.addRole("role1", new HashSet<String>());
+    assertSame(role, authPolicy.getPrincipal("role1", TPrincipalType.ROLE));
+    // Delete the role.
+    assertSame(role, catalog_.removeRole("role1"));
+    assertNull(authPolicy.getPrincipal("role1", TPrincipalType.ROLE));
+
+    // Assert that principal IDs will be unique between roles and users, e.g. no user and
+    // role with the same principal ID. The same name can be used for both user and role.
+    int size = 10;
+    String prefix = "foo";
+    for (int i = 0; i < size; i++) {
+      String name = prefix + i;
+      catalog_.addUser(name);
+      catalog_.addRole(name, new HashSet<String>());
+    }
+
+    for (int i = 0; i < size; i++) {
+      String name = prefix + i;
+      Principal u = authPolicy.getPrincipal(name, TPrincipalType.USER);
+      Principal r = authPolicy.getPrincipal(name, TPrincipalType.ROLE);
+      assertEquals(name, u.getName());
+      assertEquals(name, r.getName());
+      assertNotEquals(u.getId(), r.getId());
+    }
+
+    // Validate getAllUsers vs getAllUserNames
+    List<User> allUsers = authPolicy.getAllUsers();
+    Set<String> allUserNames = authPolicy.getAllUserNames();
+    assertEquals(allUsers.size(), allUserNames.size());
+    for (Principal principal: allUsers) {
+      assertTrue(allUserNames.contains(principal.getName()));
+    }
+
+    // Validate getAllRoles and getAllRoleNames work as expected.
+    List<Role> allRoles = authPolicy.getAllRoles();
+    Set<String> allRoleNames = authPolicy.getAllRoleNames();
+    assertEquals(allRoles.size(), allRoleNames.size());
+    for (Principal principal: allRoles) {
+      assertTrue(allRoleNames.contains(principal.getName()));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/a23e6f29/fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java b/fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
index 2b2fdf8..3186113 100644
--- a/fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
+++ b/fe/src/test/java/org/apache/impala/testutil/ImpaladTestCatalog.java
@@ -27,9 +27,11 @@ import org.apache.impala.catalog.FeDb;
 import org.apache.impala.catalog.HdfsCachePool;
 import org.apache.impala.catalog.HdfsTable;
 import org.apache.impala.catalog.ImpaladCatalog;
+import org.apache.impala.catalog.Principal;
+import org.apache.impala.catalog.PrincipalPrivilege;
 import org.apache.impala.catalog.Role;
-import org.apache.impala.catalog.RolePrivilege;
 import org.apache.impala.catalog.Table;
+import org.apache.impala.catalog.User;
 import org.apache.impala.thrift.TPrivilege;
 import org.apache.impala.util.PatternMatcher;
 
@@ -123,14 +125,26 @@ public class ImpaladTestCatalog extends ImpaladCatalog {
     return srcCatalog_.addRole(roleName, new HashSet<String>());
   }
 
-  public Role addRoleGrantGroup(String roleName, String groupName) throws CatalogException {
+  public Role addRoleGrantGroup(String roleName, String groupName)
+      throws CatalogException {
     return srcCatalog_.addRoleGrantGroup(roleName, groupName);
   }
 
-  public RolePrivilege addRolePrivilege(String roleName, TPrivilege privilege)
+  public PrincipalPrivilege addRolePrivilege(String roleName, TPrivilege privilege)
       throws CatalogException {
     return srcCatalog_.addRolePrivilege(roleName, privilege);
   }
 
   public void removeRole(String roleName) { srcCatalog_.removeRole(roleName); }
+
+  public User addUser(String userName) {
+    return srcCatalog_.addUser(userName);
+  }
+
+  public PrincipalPrivilege addUserPrivilege(String userName, TPrivilege privilege)
+      throws CatalogException {
+    return srcCatalog_.addUserPrivilege(userName, privilege);
+  }
+
+  public void removeUser(String userName) { srcCatalog_.removeUser(userName); }
 }