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 2019/05/12 18:34:17 UTC

[impala] 01/03: IMPALA-8281: Misc Sentry decoupling clean up

This is an automated email from the ASF dual-hosted git repository.

tarmstrong pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 6140c1c5af1c6a33804c6be8683e88c955d2a47a
Author: Austin Nobis <an...@cloudera.com>
AuthorDate: Thu May 2 15:52:59 2019 -0500

    IMPALA-8281: Misc Sentry decoupling clean up
    
    This patch moves Sentry specific code to the Sentry specific plugin
    implementation.
    
    Testing:
    - Ran all FE tests
    - Ran E2E authorization tests
    - Added new E2E tests in test_sentry
    
    Change-Id: Id24a00dd395e30e4c392f085893e9561da2ee539
    Reviewed-on: http://gerrit.cloudera.org:8080/13284
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 common/thrift/Frontend.thrift                      |  4 +-
 .../apache/impala/analysis/CreateDropRoleStmt.java |  6 ---
 .../impala/analysis/GrantRevokePrivStmt.java       | 14 +----
 .../impala/analysis/GrantRevokeRoleStmt.java       |  3 --
 .../impala/analysis/ShowGrantPrincipalStmt.java    | 24 ---------
 .../org/apache/impala/analysis/ShowRolesStmt.java  |  1 -
 .../sentry/SentryCatalogdAuthorizationManager.java | 62 +++++++++++++++++-----
 .../sentry/SentryImpaladAuthorizationManager.java  | 44 ++++++++++++++-
 .../java/org/apache/impala/service/Frontend.java   | 31 +----------
 .../impala/analysis/AnalyzeAuthStmtsTest.java      | 13 -----
 tests/authorization/test_sentry.py                 | 42 +++++++++++++++
 11 files changed, 140 insertions(+), 104 deletions(-)

diff --git a/common/thrift/Frontend.thrift b/common/thrift/Frontend.thrift
index 3f0075b..2599910 100644
--- a/common/thrift/Frontend.thrift
+++ b/common/thrift/Frontend.thrift
@@ -264,7 +264,7 @@ struct TShowRolesParams {
   // True if this opertion requires admin privileges on the Sentry Service. This is
   // needed to check for the case where an operation is_user_scope, but the user does
   // not belong to the specified grant_group.
-  2: required bool is_admin_op
+  // REMOVED: 2: required bool is_admin_op
 
   // True if the statement is "SHOW CURRENT ROLES".
   3: required bool is_show_current_roles
@@ -292,7 +292,7 @@ struct TShowGrantPrincipalParams {
 
   // True if this operation requires admin privileges on the Sentry Service (when
   // the requesting user has not been granted the target role name).
-  4: required bool is_admin_op
+  // REMOVED: 4: required bool is_admin_op
 
   // An optional filter to show grants that match a specific privilege spec.
   5: optional CatalogObjects.TPrivilege privilege
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateDropRoleStmt.java b/fe/src/main/java/org/apache/impala/analysis/CreateDropRoleStmt.java
index fb18d41..e85c0ab 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateDropRoleStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateDropRoleStmt.java
@@ -51,11 +51,5 @@ public class CreateDropRoleStmt extends AuthorizationStmt {
   @Override
   public void analyze(Analyzer analyzer) throws AnalysisException {
     super.analyze(analyzer);
-    Role existingRole = analyzer.getCatalog().getAuthPolicy().getRole(roleName_);
-    if (isDropRole_ && existingRole == null) {
-      throw new AnalysisException(String.format("Role '%s' does not exist.", roleName_));
-    } else if (!isDropRole_ && existingRole != null) {
-      throw new AnalysisException(String.format("Role '%s' already exists.", roleName_));
-    }
   }
 }
\ No newline at end of file
diff --git a/fe/src/main/java/org/apache/impala/analysis/GrantRevokePrivStmt.java b/fe/src/main/java/org/apache/impala/analysis/GrantRevokePrivStmt.java
index d2e285a..0fe898e 100644
--- a/fe/src/main/java/org/apache/impala/analysis/GrantRevokePrivStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/GrantRevokePrivStmt.java
@@ -33,8 +33,8 @@ import com.google.common.base.Strings;
  * All privilege checks on catalog objects are skipped when executing
  * GRANT/REVOKE statements. This is because we need to be able to create
  * privileges on an object before any privileges actually exist.
- * The GRANT/REVOKE statement itself will be authorized (currently by
- * the Sentry Service).
+ * The GRANT/REVOKE statement itself will be authorized by the loaded
+ * authorization provider.
  */
 public class GrantRevokePrivStmt extends AuthorizationStmt {
   private final PrivilegeSpec privilegeSpec_;
@@ -43,10 +43,6 @@ public class GrantRevokePrivStmt extends AuthorizationStmt {
   private final boolean hasGrantOpt_;
   private final TPrincipalType principalType_;
 
-  // Set/modified during analysis
-  // TODO: This will need to be cleaned up when Ranger supports RBAC
-  private Role role_;
-
   public GrantRevokePrivStmt(String roleName, PrivilegeSpec privilegeSpec,
       boolean isGrantPrivStmt, boolean hasGrantOpt, TPrincipalType principalType) {
     Preconditions.checkNotNull(privilegeSpec);
@@ -64,9 +60,6 @@ public class GrantRevokePrivStmt extends AuthorizationStmt {
     params.setIs_grant(isGrantPrivStmt_);
     List<TPrivilege> privileges = privilegeSpec_.toThrift();
     for (TPrivilege privilege: privileges) {
-      if (principalType_ == TPrincipalType.ROLE && role_ != null) {
-        privilege.setPrincipal_id(role_.getId());
-      }
       privilege.setPrincipal_type(principalType_);
       privilege.setHas_grant_opt(hasGrantOpt_);
     }
@@ -102,9 +95,6 @@ public class GrantRevokePrivStmt extends AuthorizationStmt {
           "empty.");
     }
 
-    if (principalType_ == TPrincipalType.ROLE) {
-      role_ = analyzer.getCatalog().getAuthPolicy().getRole(principalName_);
-    }
     privilegeSpec_.analyze(analyzer);
   }
 }
diff --git a/fe/src/main/java/org/apache/impala/analysis/GrantRevokeRoleStmt.java b/fe/src/main/java/org/apache/impala/analysis/GrantRevokeRoleStmt.java
index c7aef59..06d9347 100644
--- a/fe/src/main/java/org/apache/impala/analysis/GrantRevokeRoleStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/GrantRevokeRoleStmt.java
@@ -57,9 +57,6 @@ public class GrantRevokeRoleStmt extends AuthorizationStmt {
   @Override
   public void analyze(Analyzer analyzer) throws AnalysisException {
     super.analyze(analyzer);
-    if (analyzer.getCatalog().getAuthPolicy().getRole(roleName_) == null) {
-      throw new AnalysisException(String.format("Role '%s' does not exist.", roleName_));
-    }
     if (Strings.isNullOrEmpty(roleName_)) {
       throw new AnalysisException("Role name in GRANT/REVOKE ROLE cannot be empty.");
     }
diff --git a/fe/src/main/java/org/apache/impala/analysis/ShowGrantPrincipalStmt.java b/fe/src/main/java/org/apache/impala/analysis/ShowGrantPrincipalStmt.java
index 34f15b7..103dde6 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ShowGrantPrincipalStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ShowGrantPrincipalStmt.java
@@ -37,9 +37,6 @@ public class ShowGrantPrincipalStmt extends AuthorizationStmt {
   private final String name_;
   private final TPrincipalType principalType_;
 
-  // Set/modified during analysis.
-  private Principal principal_;
-
   public ShowGrantPrincipalStmt(String name, TPrincipalType principalType,
       PrivilegeSpec privilegeSpec) {
     Preconditions.checkNotNull(name);
@@ -58,24 +55,6 @@ public class ShowGrantPrincipalStmt extends AuthorizationStmt {
           Principal.toString(principalType_).toUpperCase()));
     }
 
-    switch(principalType_) {
-      case ROLE:
-        principal_ = analyzer.getCatalog().getAuthPolicy().getPrincipal(name_,
-            principalType_);
-        if (principal_ == null) {
-          throw new AnalysisException(String.format("%s '%s' " +
-              "does not exist.", Principal.toString(principalType_), name_));
-        }
-        break;
-      case USER:
-      case GROUP:
-        principal_ = Principal.newInstance(name_, principalType_, new HashSet<>());
-        break;
-      default:
-        throw new AnalysisException(String.format("Unexpected TPrincipalType %s",
-            principalType_.name()));
-    }
-
     if (privilegeSpec_ != null) privilegeSpec_.analyze(analyzer);
   }
 
@@ -118,10 +97,7 @@ public class ShowGrantPrincipalStmt extends AuthorizationStmt {
     params.setRequesting_user(requestingUser_.getShortName());
     if (privilegeSpec_ != null) {
       params.setPrivilege(privilegeSpec_.toThrift().get(0));
-      params.getPrivilege().setPrincipal_id(principal_.getId());
     }
     return params;
   }
-
-  public Principal getPrincipal() { return principal_; }
 }
diff --git a/fe/src/main/java/org/apache/impala/analysis/ShowRolesStmt.java b/fe/src/main/java/org/apache/impala/analysis/ShowRolesStmt.java
index ded1445..8d4d62f 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ShowRolesStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ShowRolesStmt.java
@@ -60,7 +60,6 @@ public class ShowRolesStmt extends AuthorizationStmt {
     params.setIs_show_current_roles(isShowCurrentRoles_);
     if (groupName_ != null) params.setGrant_group(groupName_);
     // Users should always be able to execute SHOW CURRENT ROLES.
-    params.setIs_admin_op(!isShowCurrentRoles_);
     return params;
   }
 
diff --git a/fe/src/main/java/org/apache/impala/authorization/sentry/SentryCatalogdAuthorizationManager.java b/fe/src/main/java/org/apache/impala/authorization/sentry/SentryCatalogdAuthorizationManager.java
index 0c3ca3c..ffe9000 100644
--- a/fe/src/main/java/org/apache/impala/authorization/sentry/SentryCatalogdAuthorizationManager.java
+++ b/fe/src/main/java/org/apache/impala/authorization/sentry/SentryCatalogdAuthorizationManager.java
@@ -21,9 +21,11 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.impala.authorization.AuthorizationDelta;
+import org.apache.impala.authorization.AuthorizationException;
 import org.apache.impala.authorization.AuthorizationManager;
 import org.apache.impala.authorization.User;
 import org.apache.impala.catalog.CatalogException;
+import org.apache.impala.catalog.CatalogObject;
 import org.apache.impala.catalog.CatalogServiceCatalog;
 import org.apache.impala.catalog.Principal;
 import org.apache.impala.catalog.PrincipalPrivilege;
@@ -51,6 +53,7 @@ import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.stream.Collectors;
 
 /**
  * An implementation of {@link AuthorizationManager} for Catalogd that uses Sentry.
@@ -90,7 +93,13 @@ public class SentryCatalogdAuthorizationManager implements AuthorizationManager
       TDdlExecResponse response) throws ImpalaException {
     verifySentryServiceEnabled();
 
-    Role role = sentryProxy_.createRole(requestingUser, params.getRole_name());
+    Role role = catalog_.getAuthPolicy().getRole(params.getRole_name());
+    if (role != null) {
+      throw new AuthorizationException(String.format("Role '%s' already exists.",
+          params.getRole_name()));
+    }
+
+    role = sentryProxy_.createRole(requestingUser, params.getRole_name());
     Preconditions.checkNotNull(role);
 
     TCatalogObject catalogObject = new TCatalogObject();
@@ -106,7 +115,13 @@ public class SentryCatalogdAuthorizationManager implements AuthorizationManager
       TDdlExecResponse response) throws ImpalaException {
     verifySentryServiceEnabled();
 
-    Role role = sentryProxy_.dropRole(requestingUser, params.getRole_name());
+    Role role = catalog_.getAuthPolicy().getRole(params.getRole_name());
+    if (role == null) {
+      throw new AuthorizationException(String.format("Role '%s' does not exist.",
+          params.getRole_name()));
+    }
+
+    role = sentryProxy_.dropRole(requestingUser, params.getRole_name());
     if (role == null) {
       // Nothing was removed from the catalogd's cache.
       response.result.setVersion(catalog_.getCatalogVersion());
@@ -135,15 +150,16 @@ public class SentryCatalogdAuthorizationManager implements AuthorizationManager
     Preconditions.checkArgument(!params.getGroup_names().isEmpty());
     verifySentryServiceEnabled();
 
+    if (catalog_.getAuthPolicy().getRole(params.getRole_names().get(0)) == null) {
+      throw new AuthorizationException(String.format("Role '%s' does not exist.",
+          params.getRole_names().get(0)));
+    }
+
     String roleName = params.getRole_names().get(0);
     String groupName = params.getGroup_names().get(0);
     Role role = sentryProxy_.grantRoleGroup(requestingUser, roleName, groupName);
     Preconditions.checkNotNull(role);
-    TCatalogObject catalogObject = new TCatalogObject();
-    catalogObject.setType(role.getCatalogObjectType());
-    catalogObject.setPrincipal(role.toThrift());
-    catalogObject.setCatalog_version(role.getCatalogVersion());
-    response.result.addToUpdated_catalog_objects(catalogObject);
+    response.result.addToUpdated_catalog_objects(createRoleObject(role));
     response.result.setVersion(role.getCatalogVersion());
   }
 
@@ -154,16 +170,26 @@ public class SentryCatalogdAuthorizationManager implements AuthorizationManager
     Preconditions.checkArgument(!params.getGroup_names().isEmpty());
     verifySentryServiceEnabled();
 
+    if (catalog_.getAuthPolicy().getRole(params.getRole_names().get(0)) == null) {
+      throw new AuthorizationException(String.format("Role '%s' does not exist.",
+          params.getRole_names().get(0)));
+    }
+
     String roleName = params.getRole_names().get(0);
     String groupName = params.getGroup_names().get(0);
     Role role = sentryProxy_.revokeRoleGroup(requestingUser, roleName, groupName);
+    response.result.addToUpdated_catalog_objects(createRoleObject(role));
+    response.result.setVersion(role.getCatalogVersion());
+  }
+
+  private static TCatalogObject createRoleObject(Role role) {
     Preconditions.checkNotNull(role);
+
     TCatalogObject catalogObject = new TCatalogObject();
     catalogObject.setType(role.getCatalogObjectType());
     catalogObject.setPrincipal(role.toThrift());
     catalogObject.setCatalog_version(role.getCatalogVersion());
-    response.result.addToUpdated_catalog_objects(catalogObject);
-    response.result.setVersion(role.getCatalogVersion());
+    return catalogObject;
   }
 
   @Override
@@ -174,11 +200,13 @@ public class SentryCatalogdAuthorizationManager implements AuthorizationManager
     String roleName = params.getPrincipal_name();
     Role role = catalog_.getAuthPolicy().getRole(roleName);
     if (role == null) {
-      throw new InternalException(String.format("Role '%s' does not exists.",
+      throw new AuthorizationException(String.format("Role '%s' does not exist.",
           roleName));
     }
 
-    List<TPrivilege> privileges = params.getPrivileges();
+    List<TPrivilege> privileges = params.getPrivileges().stream()
+        .peek(p -> p.setPrincipal_id(role.getId()))
+        .collect(Collectors.toList());
     List<PrincipalPrivilege> removedGrantOptPrivileges =
         Lists.newArrayListWithExpectedSize(privileges.size());
     List<PrincipalPrivilege> addedRolePrivileges =
@@ -216,8 +244,18 @@ public class SentryCatalogdAuthorizationManager implements AuthorizationManager
     verifySentryServiceEnabled();
     Preconditions.checkArgument(!params.getPrivileges().isEmpty());
 
+    Role role = catalog_.getAuthPolicy().getRole(params.principal_name);
+    if (role == null) {
+      throw new AuthorizationException(String.format("Role '%s' does not exist.",
+          params.getPrincipal_name()));
+    }
+
     String roleName = params.getPrincipal_name();
-    List<TPrivilege> privileges = params.getPrivileges();
+    List<TPrivilege> privileges = params.getPrivileges().stream()
+        .peek(p -> {
+          if (role != null) p.setPrincipal_id(role.getId());
+        }).collect(Collectors.toList());
+
     // If this is a revoke of a privilege that contains the grant option, the privileges
     // with the grant option will be revoked and new privileges without the grant option
     // will be added.  The privilege in the catalog cannot simply be updated since the
diff --git a/fe/src/main/java/org/apache/impala/authorization/sentry/SentryImpaladAuthorizationManager.java b/fe/src/main/java/org/apache/impala/authorization/sentry/SentryImpaladAuthorizationManager.java
index 92c13b3..e4bdfba 100644
--- a/fe/src/main/java/org/apache/impala/authorization/sentry/SentryImpaladAuthorizationManager.java
+++ b/fe/src/main/java/org/apache/impala/authorization/sentry/SentryImpaladAuthorizationManager.java
@@ -27,6 +27,7 @@ import org.apache.impala.authorization.AuthorizationDelta;
 import org.apache.impala.authorization.AuthorizationException;
 import org.apache.impala.authorization.AuthorizationManager;
 import org.apache.impala.authorization.User;
+import org.apache.impala.catalog.Principal;
 import org.apache.impala.catalog.Role;
 import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.InternalException;
@@ -50,10 +51,13 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.function.Supplier;
 
+import static org.apache.impala.thrift.TPrincipalType.ROLE;
+
 /**
  * An implementation of {@link AuthorizationManager} for Impalad that uses Sentry.
  *
@@ -95,7 +99,18 @@ public class SentryImpaladAuthorizationManager implements AuthorizationManager {
 
   @Override
   public TShowRolesResult getRoles(TShowRolesParams params) throws ImpalaException {
-    if (params.isIs_admin_op()) {
+    Set<String> groups = authzChecker_.get()
+        .getUserGroups(new User(params.requesting_user));
+
+    // Check if the user is part of the group (case-sensitive) this SHOW ROLES
+    // statement is targeting. If they are already a member of the group,
+    // the admin requirement can be removed.
+    // If the the statement is SHOW CURRENT ROLES, the admin requirement can also be
+    // removed.
+    boolean adminOp =
+        !(groups.contains(params.getGrant_group()) || params.is_show_current_roles);
+
+    if (adminOp) {
       validateSentryAdmin(params.getRequesting_user());
     }
 
@@ -187,7 +202,17 @@ public class SentryImpaladAuthorizationManager implements AuthorizationManager {
   @Override
   public TResultSet getPrivileges(TShowGrantPrincipalParams params)
       throws ImpalaException {
-    if (params.isIs_admin_op()) {
+    Principal principal = (params.principal_type == ROLE) ?
+        catalog_.getOrCreateCatalog().getAuthPolicy().getPrincipal(params.getName(),
+            params.getPrincipal_type()) :
+        Principal.newInstance(params.name, params.principal_type, new HashSet<>());
+
+    if (principal == null) {
+      throw new AuthorizationException(String.format("%s '%s' does not exist.",
+          Principal.toString(params.principal_type), params.name));
+    }
+
+    if (isAdminOp(params, principal, authzChecker_.get())) {
       validateSentryAdmin(params.getRequesting_user());
     }
 
@@ -206,6 +231,21 @@ public class SentryImpaladAuthorizationManager implements AuthorizationManager {
     }
   }
 
+  private static boolean isAdminOp(TShowGrantPrincipalParams params, Principal principal,
+      SentryAuthorizationChecker authzChecker) throws ImpalaException {
+    Set<String> groupNames = authzChecker.getUserGroups(new User(params.requesting_user));
+
+    switch (params.principal_type) {
+      case USER:
+        return !principal.getName().equals(params.requesting_user);
+      case GROUP:
+      case ROLE:
+        return Sets.intersection(groupNames, principal.getGrantGroups()).isEmpty();
+      default:
+        return false;
+    }
+  }
+
   @Override
   public void updateDatabaseOwnerPrivilege(String serverName, String databaseName,
       String oldOwner, PrincipalType oldOwnerType, String newOwner,
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index cb9f9ab..1f5c803 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -499,24 +499,12 @@ public class Frontend {
       ResetMetadataStmt resetMetadataStmt = (ResetMetadataStmt) analysis.getStmt();
       TResetMetadataRequest req = resetMetadataStmt.toThrift();
       ddl.setReset_metadata_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
+      metadata.setColumns(Collections.emptyList());
     } else if (analysis.isShowRolesStmt()) {
       ddl.op_type = TCatalogOpType.SHOW_ROLES;
       ShowRolesStmt showRolesStmt = (ShowRolesStmt) analysis.getStmt();
       ddl.setShow_roles_params(showRolesStmt.toThrift());
       Preconditions.checkState(getAuthzChecker() instanceof SentryAuthorizationChecker);
-      Set<String> groupNames = getAuthzChecker().getUserGroups(
-          analysis.getAnalyzer().getUser());
-      // Check if the user is part of the group (case-sensitive) this SHOW ROLE
-      // statement is targeting. If they are already a member of the group,
-      // the admin requirement can be removed.
-      // If the the statement is SHOW CURRENT ROLES, the admin requirement can also be
-      // removed.
-      Preconditions.checkState(ddl.getShow_roles_params().isSetIs_admin_op());
-      ddl.getShow_roles_params().setIs_admin_op(!(
-          (ddl.getShow_roles_params().isSetGrant_group() &&
-              groupNames.contains(ddl.getShow_roles_params().getGrant_group())) ||
-              ddl.getShow_roles_params().isIs_show_current_roles()));
       metadata.setColumns(Arrays.asList(
           new TColumn("role_name", Type.STRING.toThrift())));
     } else if (analysis.isShowGrantPrincipalStmt()) {
@@ -524,22 +512,7 @@ public class Frontend {
       ShowGrantPrincipalStmt showGrantPrincipalStmt =
           (ShowGrantPrincipalStmt) analysis.getStmt();
       ddl.setShow_grant_principal_params(showGrantPrincipalStmt.toThrift());
-      Set<String> groupNames = getAuthzChecker().getUserGroups(
-          analysis.getAnalyzer().getUser());
-      // User must be an admin to execute this operation if they have not been granted
-      // this principal, or the same user as the request.
-      boolean requiresAdmin;
-      if (showGrantPrincipalStmt.getPrincipal().getPrincipalType()
-          == TPrincipalType.USER) {
-        requiresAdmin = !showGrantPrincipalStmt.getPrincipal().getName().equals(
-            analysis.getAnalyzer().getUser().getShortName());
-      } else {
-        requiresAdmin = Sets.intersection(groupNames, showGrantPrincipalStmt
-            .getPrincipal().getGrantGroups()).isEmpty();
-      }
-      ddl.getShow_grant_principal_params().setIs_admin_op(requiresAdmin);
-      metadata.setColumns(Arrays.asList(
-          new TColumn("name", Type.STRING.toThrift())));
+      metadata.setColumns(Arrays.asList(new TColumn("name", Type.STRING.toThrift())));
     } else if (analysis.isCreateDropRoleStmt()) {
       CreateDropRoleStmt createDropRoleStmt = (CreateDropRoleStmt) analysis.getStmt();
       TCreateDropRoleParams params = createDropRoleStmt.toThrift();
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 e3ce330..1532965 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
@@ -116,11 +116,6 @@ public class AnalyzeAuthStmtsTest extends FrontendTestBase {
       AnalysisError("SHOW GRANT ROLE myRole ON SERVER", authDisabledCtx,
           "Authorization is not enabled.");
     }
-    AnalysisError("SHOW GRANT ROLE does_not_exist",
-        "Role 'does_not_exist' does not exist.");
-
-    AnalysisError("SHOW GRANT ROLE does_not_exist ON SERVER",
-        "Role 'does_not_exist' does not exist.");
 
     // Determining if a user exists on the system is done in the AuthorizationPolicy and
     // these tests run with authorization disabled. The SHOW GRANT USER will be tested
@@ -132,12 +127,8 @@ public class AnalyzeAuthStmtsTest extends FrontendTestBase {
     AnalyzesOk("DROP ROLE myRole");
     AnalyzesOk("CREATE ROLE doesNotExist");
 
-    AnalysisError("DROP ROLE doesNotExist", "Role 'doesNotExist' does not exist.");
-    AnalysisError("CREATE ROLE myRole", "Role 'myRole' already exists.");
-
     // Role names are case-insensitive
     AnalyzesOk("DROP ROLE MYrole");
-    AnalysisError("CREATE ROLE MYrole", "Role 'MYrole' already exists.");
 
     AnalysisContext authDisabledCtx = createAuthDisabledAnalysisCtx();
     AnalysisError("DROP ROLE myRole", authDisabledCtx,
@@ -150,10 +141,6 @@ public class AnalyzeAuthStmtsTest extends FrontendTestBase {
   public void AnalyzeGrantRevokeRole() throws AnalysisException {
     AnalyzesOk("GRANT ROLE myrole TO GROUP abc");
     AnalyzesOk("REVOKE ROLE myrole FROM GROUP abc");
-    AnalysisError("GRANT ROLE doesNotExist TO GROUP abc",
-        "Role 'doesNotExist' does not exist.");
-    AnalysisError("REVOKE ROLE doesNotExist FROM GROUP abc",
-        "Role 'doesNotExist' does not exist.");
 
     AnalysisContext authDisabledCtx = createAuthDisabledAnalysisCtx();
     AnalysisError("GRANT ROLE myrole TO GROUP abc", authDisabledCtx,
diff --git a/tests/authorization/test_sentry.py b/tests/authorization/test_sentry.py
index 494fdb5..f650b1d 100644
--- a/tests/authorization/test_sentry.py
+++ b/tests/authorization/test_sentry.py
@@ -15,11 +15,13 @@
 # specific language governing permissions and limitations
 # under the License.
 
+import grp
 import pytest
 import os
 from getpass import getuser
 
 from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
+from tests.common.sentry_cache_test_suite import SentryCacheTestSuite
 
 SENTRY_CONFIG_DIR = os.getenv('IMPALA_HOME') + '/fe/src/test/resources/'
 SENTRY_CONFIG_FILE = SENTRY_CONFIG_DIR + 'sentry-site.xml'
@@ -58,3 +60,43 @@ class TestSentry(CustomClusterTestSuite):
                "the requested policy metadata.".format(non_admin) in str(result)
     finally:
       admin_client.execute("drop role {0}".format(unique_role))
+
+  @pytest.mark.execute_serially
+  @SentryCacheTestSuite.with_args(
+    impalad_args="--server_name=server1",
+    catalogd_args="--sentry_config=" + SENTRY_CONFIG_FILE,
+    sentry_config=SENTRY_CONFIG_FILE)
+  def test_grant_revoke_invalid_role(self, unique_name):
+    role_name = "foobar"
+    group = grp.getgrnam(getuser()).gr_name
+    try:
+      # This will create "foobar" role catalog object.
+      self.client.execute("create role {0}".format(role_name))
+      self.client.execute("grant all on server to {0}".format(role_name))
+      self.client.execute("grant role {0} to group `{1}`".format(role_name, group))
+      self.client.execute("create database {0}".format(unique_name))
+
+      ex = self.execute_query_expect_failure(
+        self.client, "grant all on database {0} to role non_role".format(unique_name))
+      assert "Role 'non_role' does not exist." in str(ex)
+
+      ex = self.execute_query_expect_failure(
+        self.client, "revoke all on database {0} from role non_role".format(unique_name))
+      assert "Role 'non_role' does not exist." in str(ex)
+
+      ex = self.execute_query_expect_failure(self.client, "show grant role non_role")
+      assert "Role 'non_role' does not exist." in str(ex)
+
+      ex = self.execute_query_expect_failure(
+        self.client, "grant role non_role to group `{0}`".format(group))
+      assert "Role 'non_role' does not exist." in str(ex)
+
+      ex = self.execute_query_expect_failure(self.client, "drop role non_role")
+      assert "Role 'non_role' does not exist." in str(ex)
+
+      ex = self.execute_query_expect_failure(self.client,
+                                             "create role {0}".format(role_name))
+      assert "Role '{0}' already exists.".format(role_name) in str(ex)
+    finally:
+      self.client.execute("drop database {0}".format(unique_name))
+      self.client.execute("drop role {0}".format(role_name))