You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ar...@apache.org on 2019/04/11 18:25:13 UTC

[impala] 01/06: IMPALA-8385: Refactor Sentry admin user check

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

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

commit 284a9814aad89877fe6b80456c628cdaaa286b24
Author: Fredy Wijaya <fw...@cloudera.com>
AuthorDate: Mon Apr 8 17:00:16 2019 -0700

    IMPALA-8385: Refactor Sentry admin user check
    
    This patch moves the hardcoded Sentry admin user check from the generic
    "show roles" and "show grant" catalog operations to Sentry authorization
    plugin, i.e. SentryImpaladAuthorizationManager. This patch also removes
    isAdmin() from AuthorizationManager interface.
    
    Testing:
    - Added a new authorization E2E test to test for Sentry admin check
    - Ran all FE tests
    - Ran all authorization E2E tests
    
    Change-Id: I911228b09af7eed5d5dc002b20591ef64dc625d3
    Reviewed-on: http://gerrit.cloudera.org:8080/12963
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/catalog/catalog-server.cc                   |  2 +-
 be/src/catalog/catalog.cc                          |  7 +--
 be/src/catalog/catalog.h                           |  7 +--
 be/src/exec/catalog-op-executor.cc                 |  8 +--
 be/src/exec/catalog-op-executor.h                  |  9 ++--
 be/src/service/client-request-state.cc             | 24 ---------
 be/src/service/fe-support.cc                       | 29 +++++++++++
 common/thrift/CatalogService.thrift                |  6 ++-
 .../impala/authorization/AuthorizationManager.java |  6 ---
 .../authorization/NoneAuthorizationFactory.java    |  5 --
 .../ranger/RangerCatalogdAuthorizationManager.java |  5 --
 .../sentry/SentryCatalogdAuthorizationManager.java |  6 ++-
 .../sentry/SentryImpaladAuthorizationManager.java  | 55 +++++++++++++++++---
 .../java/org/apache/impala/service/FeSupport.java  | 11 ++++
 .../java/org/apache/impala/service/Frontend.java   | 12 ++---
 .../java/org/apache/impala/service/JniCatalog.java | 24 +++++----
 tests/authorization/test_sentry.py                 | 60 ++++++++++++++++++++++
 17 files changed, 194 insertions(+), 82 deletions(-)

diff --git a/be/src/catalog/catalog-server.cc b/be/src/catalog/catalog-server.cc
index 7952699..dee1871 100644
--- a/be/src/catalog/catalog-server.cc
+++ b/be/src/catalog/catalog-server.cc
@@ -208,7 +208,7 @@ class CatalogServiceThriftIf : public CatalogServiceIf {
   void SentryAdminCheck(TSentryAdminCheckResponse& resp,
       const TSentryAdminCheckRequest& req) override {
     VLOG_RPC << "SentryAdminCheck(): request=" << ThriftDebugString(req);
-    Status status = catalog_server_->catalog()->SentryAdminCheck(req);
+    Status status = catalog_server_->catalog()->SentryAdminCheck(req, &resp);
     if (!status.ok()) LOG(ERROR) << status.GetDetail();
     TStatus thrift_status;
     status.ToThrift(&thrift_status);
diff --git a/be/src/catalog/catalog.cc b/be/src/catalog/catalog.cc
index c6becc3..585234a 100644
--- a/be/src/catalog/catalog.cc
+++ b/be/src/catalog/catalog.cc
@@ -61,7 +61,7 @@ Catalog::Catalog() {
     {"getTableMetrics", "([B)Ljava/lang/String;", &get_table_metrics_id_},
     {"getDbs", "([B)[B", &get_dbs_id_},
     {"getFunctions", "([B)[B", &get_functions_id_},
-    {"checkUserSentryAdmin", "([B)V", &sentry_admin_check_id_},
+    {"checkUserSentryAdmin", "([B)[B", &sentry_admin_check_id_},
     {"getCatalogObject", "([B)[B", &get_catalog_object_id_},
     {"getPartialCatalogObject", "([B)[B", &get_partial_catalog_object_id_},
     {"getCatalogDelta", "([B)[B", &get_catalog_delta_id_},
@@ -184,8 +184,9 @@ Status Catalog::GetPartitionStats(
   return JniUtil::CallJniMethod(catalog_, get_partition_stats_id_, req, resp);
 }
 
-Status Catalog::SentryAdminCheck(const TSentryAdminCheckRequest& req) {
-  return JniUtil::CallJniMethod(catalog_, sentry_admin_check_id_, req);
+Status Catalog::SentryAdminCheck(const TSentryAdminCheckRequest& req,
+    TSentryAdminCheckResponse* resp) {
+  return JniUtil::CallJniMethod(catalog_, sentry_admin_check_id_, req, resp);
 }
 
 Status Catalog::UpdateTableUsage(const TUpdateTableUsageRequest& req) {
diff --git a/be/src/catalog/catalog.h b/be/src/catalog/catalog.h
index 4096732..f1fe2e9 100644
--- a/be/src/catalog/catalog.h
+++ b/be/src/catalog/catalog.h
@@ -126,9 +126,10 @@ class Catalog {
       const TGetPartitionStatsRequest& req, TGetPartitionStatsResponse* resp);
 
   /// Checks whether the requesting user has admin privileges on the Sentry Service and
-  /// returns OK if they do. Returns a bad status if the user is not an admin or if there
-  /// was an error executing the request.
-  Status SentryAdminCheck(const TSentryAdminCheckRequest& req);
+  /// returns OK if they do. Returns a bad status if there was an error executing the
+  /// request.
+  Status SentryAdminCheck(const TSentryAdminCheckRequest& req,
+      TSentryAdminCheckResponse* resp);
 
   /// Update recently used table names and their use counts in an impalad since the last
   /// report.
diff --git a/be/src/exec/catalog-op-executor.cc b/be/src/exec/catalog-op-executor.cc
index ce564d8..e820b0a 100644
--- a/be/src/exec/catalog-op-executor.cc
+++ b/be/src/exec/catalog-op-executor.cc
@@ -324,16 +324,16 @@ Status CatalogOpExecutor::GetPartitionStats(
   return Status::OK();
 }
 
-Status CatalogOpExecutor::SentryAdminCheck(const TSentryAdminCheckRequest& req) {
+Status CatalogOpExecutor::SentryAdminCheck(const TSentryAdminCheckRequest& req,
+    TSentryAdminCheckResponse* result) {
   const TNetworkAddress& address =
       MakeNetworkAddress(FLAGS_catalog_service_host, FLAGS_catalog_service_port);
   Status cnxn_status;
   CatalogServiceConnection client(env_->catalogd_client_cache(), address, &cnxn_status);
   RETURN_IF_ERROR(cnxn_status);
-  TSentryAdminCheckResponse resp;
   RETURN_IF_ERROR(
-      client.DoRpc(&CatalogServiceClientWrapper::SentryAdminCheck, req, &resp));
-  return Status(resp.status);
+      client.DoRpc(&CatalogServiceClientWrapper::SentryAdminCheck, req, result));
+  return Status::OK();
 }
 
 Status CatalogOpExecutor::UpdateTableUsage(const TUpdateTableUsageRequest& req,
diff --git a/be/src/exec/catalog-op-executor.h b/be/src/exec/catalog-op-executor.h
index 15d5c5b..8284d85 100644
--- a/be/src/exec/catalog-op-executor.h
+++ b/be/src/exec/catalog-op-executor.h
@@ -79,10 +79,11 @@ class CatalogOpExecutor {
   Status UpdateTableUsage(const TUpdateTableUsageRequest& req,
       TUpdateTableUsageResponse* resp);
 
-  /// Makes an RPC to the CatalogServer to verify whether the specified user has privileges
-  /// to access the Sentry Policy Service. Returns OK if the user has privileges or
-  /// a bad status if the user does not have privileges (or if there was an error).
-  Status SentryAdminCheck(const TSentryAdminCheckRequest& re);
+  /// Makes an RPC to the CatalogServer to verify whether the specified user has
+  /// privileges / to access the Sentry Policy Service. Returns OK if the RPC was
+  /// successful, otherwise a bad status will be returned.
+  Status SentryAdminCheck(const TSentryAdminCheckRequest& req,
+      TSentryAdminCheckResponse* resp);
 
   /// Set in Exec(), returns a pointer to the TDdlExecResponse of the DDL execution.
   /// If called before Exec(), this will return NULL. Only set if the
diff --git a/be/src/service/client-request-state.cc b/be/src/service/client-request-state.cc
index f8c1b95..5e667fd 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -354,18 +354,6 @@ Status ClientRequestState::ExecLocalCatalogOp(
     }
     case TCatalogOpType::SHOW_ROLES: {
       const TShowRolesParams& params = catalog_op.show_roles_params;
-      if (params.is_admin_op) {
-        // Verify the user has privileges to perform this operation by checking against
-        // the Sentry Service (via the Catalog Server).
-        catalog_op_executor_.reset(new CatalogOpExecutor(exec_env_, frontend_,
-            server_profile_));
-
-        TSentryAdminCheckRequest req;
-        req.__set_header(TCatalogServiceRequestHeader());
-        req.header.__set_requesting_user(effective_user());
-        RETURN_IF_ERROR(catalog_op_executor_->SentryAdminCheck(req));
-      }
-
       // If we have made it here, the user has privileges to execute this operation.
       // Return the results.
       TShowRolesResult result;
@@ -375,18 +363,6 @@ Status ClientRequestState::ExecLocalCatalogOp(
     }
     case TCatalogOpType::SHOW_GRANT_PRINCIPAL: {
       const TShowGrantPrincipalParams& params = catalog_op.show_grant_principal_params;
-      if (params.is_admin_op) {
-        // Verify the user has privileges to perform this operation by checking against
-        // the Sentry Service (via the Catalog Server).
-        catalog_op_executor_.reset(new CatalogOpExecutor(exec_env_, frontend_,
-            server_profile_));
-
-        TSentryAdminCheckRequest req;
-        req.__set_header(TCatalogServiceRequestHeader());
-        req.header.__set_requesting_user(effective_user());
-        RETURN_IF_ERROR(catalog_op_executor_->SentryAdminCheck(req));
-      }
-
       TResultSet response;
       RETURN_IF_ERROR(frontend_->GetPrincipalPrivileges(params, &response));
       // Set the result set and its schema from the response.
diff --git a/be/src/service/fe-support.cc b/be/src/service/fe-support.cc
index 4a25250..5f16b86 100644
--- a/be/src/service/fe-support.cc
+++ b/be/src/service/fe-support.cc
@@ -549,6 +549,30 @@ Java_org_apache_impala_service_FeSupport_NativeUpdateTableUsage(
   return result_bytes;
 }
 
+// Calls the catalog server to to check if the given user is a Sentry admin.
+extern "C"
+JNIEXPORT jbyteArray JNICALL
+Java_org_apache_impala_service_FeSupport_NativeSentryAdminCheck(
+    JNIEnv* env, jclass caller_class, jbyteArray thrift_struct) {
+  TSentryAdminCheckRequest request;
+  THROW_IF_ERROR_RET(DeserializeThriftMsg(env, thrift_struct, &request), env,
+      JniUtil::internal_exc_class(), nullptr);
+
+  CatalogOpExecutor catalog_op_executor(ExecEnv::GetInstance(), nullptr, nullptr);
+  TSentryAdminCheckResponse result;
+  Status status = catalog_op_executor.SentryAdminCheck(request, &result);
+  if (!status.ok()) {
+    LOG(ERROR) << status.GetDetail();
+    status.AddDetail("Error making an RPC call to Catalog server.");
+    status.SetTStatus(&result);
+  }
+
+  jbyteArray result_bytes = nullptr;
+  THROW_IF_ERROR_RET(SerializeThriftMsg(env, &result, &result_bytes), env,
+      JniUtil::internal_exc_class(), result_bytes);
+  return result_bytes;
+}
+
 // Calls in to the catalog server to request partial information about a
 // catalog object.
 extern "C"
@@ -663,6 +687,11 @@ static JNINativeMethod native_methods[] = {
       (void*)::Java_org_apache_impala_service_FeSupport_NativeUpdateTableUsage
   },
   {
+      const_cast<char*>("NativeSentryAdminCheck"),
+      const_cast<char*>("([B)[B"),
+      (void*)::Java_org_apache_impala_service_FeSupport_NativeSentryAdminCheck
+  },
+  {
       const_cast<char*>("NativeParseQueryOptions"),
       const_cast<char*>("(Ljava/lang/String;[B)[B"),
       (void*)::Java_org_apache_impala_service_FeSupport_NativeParseQueryOptions
diff --git a/common/thrift/CatalogService.thrift b/common/thrift/CatalogService.thrift
index 0f186a2..cd5b775 100644
--- a/common/thrift/CatalogService.thrift
+++ b/common/thrift/CatalogService.thrift
@@ -478,9 +478,11 @@ struct TSentryAdminCheckRequest {
 }
 
 struct TSentryAdminCheckResponse {
-  // Contains an error if the user does not have privileges to access the Sentry Service
-  // or if the Sentry Service is unavailable. Returns OK if the operation was successful.
+  // Returns OK if the operation was successful.
   1: optional Status.TStatus status
+
+  // Returns true if the user is a Sentry admin user.
+  2: required bool is_admin
 }
 
 struct TTableUsage {
diff --git a/fe/src/main/java/org/apache/impala/authorization/AuthorizationManager.java b/fe/src/main/java/org/apache/impala/authorization/AuthorizationManager.java
index 0f42345..e8ad5b9 100644
--- a/fe/src/main/java/org/apache/impala/authorization/AuthorizationManager.java
+++ b/fe/src/main/java/org/apache/impala/authorization/AuthorizationManager.java
@@ -36,12 +36,6 @@ import org.apache.impala.thrift.TShowRolesResult;
  */
 public interface AuthorizationManager {
   /**
-   * Returns true if the given user is an admin user.
-   * TODO: Do we really need this?
-   */
-  boolean isAdmin(User user) throws ImpalaException;
-
-  /**
    * Creates a role.
    */
   void createRole(User requestingUser, TCreateDropRoleParams params,
diff --git a/fe/src/main/java/org/apache/impala/authorization/NoneAuthorizationFactory.java b/fe/src/main/java/org/apache/impala/authorization/NoneAuthorizationFactory.java
index f21752c..f347b60 100644
--- a/fe/src/main/java/org/apache/impala/authorization/NoneAuthorizationFactory.java
+++ b/fe/src/main/java/org/apache/impala/authorization/NoneAuthorizationFactory.java
@@ -72,11 +72,6 @@ public class NoneAuthorizationFactory implements AuthorizationFactory {
 
   public static class NoneAuthorizationManager implements AuthorizationManager {
     @Override
-    public boolean isAdmin(User user) throws ImpalaException {
-      return false;
-    }
-
-    @Override
     public void createRole(User requestingUser, TCreateDropRoleParams params,
         TDdlExecResponse response) throws ImpalaException {
       throw new UnsupportedOperationException(String.format("%s is not supported",
diff --git a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerCatalogdAuthorizationManager.java b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerCatalogdAuthorizationManager.java
index cedd1e8..22c4b04 100644
--- a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerCatalogdAuthorizationManager.java
+++ b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerCatalogdAuthorizationManager.java
@@ -54,11 +54,6 @@ public class RangerCatalogdAuthorizationManager implements AuthorizationManager
   }
 
   @Override
-  public boolean isAdmin(User user) throws ImpalaException {
-    return false;
-  }
-
-  @Override
   public void createRole(User requestingUser, TCreateDropRoleParams params,
       TDdlExecResponse response) throws ImpalaException {
     throw new UnsupportedOperationException(String.format(
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 872dc63..ed9108c 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
@@ -71,8 +71,10 @@ public class SentryCatalogdAuthorizationManager implements AuthorizationManager
     catalog_ = catalog;
   }
 
-  @Override
-  public boolean isAdmin(User user) throws ImpalaException {
+  /**
+   * Checks if the given user is a Sentry admin.
+   */
+  public boolean isSentryAdmin(User user) throws ImpalaException {
     return catalog_.getSentryProxy().isSentryAdmin(user);
   }
 
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 73cc11c..932a267 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
@@ -17,22 +17,30 @@
 
 package org.apache.impala.authorization.sentry;
 
+import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
 import org.apache.impala.authorization.AuthorizationChecker;
+import org.apache.impala.authorization.AuthorizationException;
 import org.apache.impala.authorization.AuthorizationManager;
 import org.apache.impala.authorization.User;
 import org.apache.impala.catalog.Role;
 import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.InternalException;
+import org.apache.impala.common.JniUtil;
 import org.apache.impala.service.FeCatalogManager;
+import org.apache.impala.service.FeSupport;
+import org.apache.impala.thrift.TCatalogServiceRequestHeader;
 import org.apache.impala.thrift.TCreateDropRoleParams;
 import org.apache.impala.thrift.TDdlExecResponse;
+import org.apache.impala.thrift.TErrorCode;
 import org.apache.impala.thrift.TGrantRevokePrivParams;
 import org.apache.impala.thrift.TGrantRevokeRoleParams;
 import org.apache.impala.thrift.TResultSet;
+import org.apache.impala.thrift.TSentryAdminCheckRequest;
+import org.apache.impala.thrift.TSentryAdminCheckResponse;
 import org.apache.impala.thrift.TShowGrantPrincipalParams;
 import org.apache.impala.thrift.TShowRolesParams;
 import org.apache.impala.thrift.TShowRolesResult;
@@ -71,12 +79,6 @@ public class SentryImpaladAuthorizationManager implements AuthorizationManager {
   }
 
   @Override
-  public boolean isAdmin(User user) throws ImpalaException {
-    throw new UnsupportedOperationException(String.format(
-        "%s is not supported in Impalad", ClassUtil.getMethodName()));
-  }
-
-  @Override
   public void createRole(User requestingUser, TCreateDropRoleParams params,
       TDdlExecResponse response) throws ImpalaException {
     throw new UnsupportedOperationException(String.format(
@@ -92,6 +94,10 @@ public class SentryImpaladAuthorizationManager implements AuthorizationManager {
 
   @Override
   public TShowRolesResult getRoles(TShowRolesParams params) throws ImpalaException {
+    if (params.isIs_admin_op()) {
+      validateSentryAdmin(params.getRequesting_user());
+    }
+
     TShowRolesResult result = new TShowRolesResult();
     List<Role> roles = Lists.newArrayList();
     if (params.isIs_show_current_roles() || params.isSetGrant_group()) {
@@ -180,6 +186,10 @@ public class SentryImpaladAuthorizationManager implements AuthorizationManager {
   @Override
   public TResultSet getPrivileges(TShowGrantPrincipalParams params)
       throws ImpalaException {
+    if (params.isIs_admin_op()) {
+      validateSentryAdmin(params.getRequesting_user());
+    }
+
     switch (params.getPrincipal_type()) {
       case USER:
         Set<String> groupNames = authzChecker_.get().getUserGroups(
@@ -210,4 +220,37 @@ public class SentryImpaladAuthorizationManager implements AuthorizationManager {
     throw new UnsupportedOperationException(String.format(
         "%s is not supported in Impalad", ClassUtil.getMethodName()));
   }
+
+  /**
+   * Validates if the given user is a Sentry admin. The Sentry admin check will make an
+   * RPC call to the Catalog server. This check is necessary because some operations
+   * in this class does not need to make a call to Sentry, e.g. "show roles" and
+   * "show grant" because the authorization data can be retrieved directly from the
+   * Impalad catalog without going to Sentry. In order to ensure those operations can
+   * only be executed by a Sentry admin, a separate call to the Catalog server is needed
+   * to check if the given user is a Sentry admin.
+   *
+   * @throws AuthorizationException thrown when a given user is not a Sentry admin.
+   */
+  private static void validateSentryAdmin(String user) throws ImpalaException {
+    TSentryAdminCheckRequest request = new TSentryAdminCheckRequest();
+    TCatalogServiceRequestHeader header = new TCatalogServiceRequestHeader();
+    header.setRequesting_user(user);
+    request.setHeader(header);
+
+    byte[] thriftReq = JniUtil.serializeToThrift(request);
+    byte[] thriftRes = FeSupport.CheckSentryAdmin(thriftReq);
+
+    TSentryAdminCheckResponse response = new TSentryAdminCheckResponse();
+    JniUtil.deserializeThrift(response, thriftRes);
+
+    if (response.getStatus().getStatus_code() != TErrorCode.OK) {
+      throw new InternalException(String.format("Error requesting SentryAdminCheck: %s",
+          Joiner.on("\n").join(response.getStatus().getError_msgs())));
+    }
+    if (!response.isIs_admin()) {
+      throw new AuthorizationException(String.format("User '%s' does not have " +
+          "privileges to access the requested policy metadata.", user));
+    }
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/service/FeSupport.java b/fe/src/main/java/org/apache/impala/service/FeSupport.java
index 36ef823..685b509 100644
--- a/fe/src/main/java/org/apache/impala/service/FeSupport.java
+++ b/fe/src/main/java/org/apache/impala/service/FeSupport.java
@@ -114,6 +114,9 @@ public class FeSupport {
 
   public native static byte[] NativeUpdateTableUsage(byte[] thriftReq);
 
+  // Does an RPC to the Catalog Server to check if the given user is a Sentry admin.
+  public native static byte[] NativeSentryAdminCheck(byte[] thriftReq);
+
   // Parses a string of comma-separated key=value query options ('csvQueryOptions'),
   // updates the existing query options ('queryOptions') with them and returns the
   // resulting serialized TQueryOptions object.
@@ -412,6 +415,14 @@ public class FeSupport {
     return NativeGetPartialCatalogObject(thriftReq);
   }
 
+  public static byte[] CheckSentryAdmin(byte[] thriftReq) {
+    try {
+      return NativeSentryAdminCheck(thriftReq);
+    } catch (UnsatisfiedLinkError e) {
+      loadLibrary();
+    }
+    return NativeSentryAdminCheck(thriftReq);
+  }
 
   /**
    * This function should be called explicitly by the FeSupport to ensure that
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 f43e959..f142245 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -24,9 +24,7 @@ import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Random;
 import java.util.Set;
-import java.util.UUID;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
@@ -509,11 +507,13 @@ public class Frontend {
       // 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());
-      if (ddl.getShow_roles_params().isSetGrant_group() &&
-          groupNames.contains(ddl.getShow_roles_params().getGrant_group())) {
-        ddl.getShow_roles_params().setIs_admin_op(false);
-      }
+      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()) {
diff --git a/fe/src/main/java/org/apache/impala/service/JniCatalog.java b/fe/src/main/java/org/apache/impala/service/JniCatalog.java
index 00c28d6..eae7fd1 100644
--- a/fe/src/main/java/org/apache/impala/service/JniCatalog.java
+++ b/fe/src/main/java/org/apache/impala/service/JniCatalog.java
@@ -26,10 +26,10 @@ import java.util.List;
 import java.util.UUID;
 
 import org.apache.impala.authorization.AuthorizationConfig;
-import org.apache.impala.authorization.AuthorizationException;
 import org.apache.impala.authorization.AuthorizationFactory;
 import org.apache.impala.authorization.NoneAuthorizationFactory;
 import org.apache.impala.authorization.User;
+import org.apache.impala.authorization.sentry.SentryCatalogdAuthorizationManager;
 import org.apache.impala.catalog.CatalogException;
 import org.apache.impala.catalog.CatalogServiceCatalog;
 import org.apache.impala.catalog.Db;
@@ -60,6 +60,7 @@ import org.apache.impala.thrift.TLogLevel;
 import org.apache.impala.thrift.TPrioritizeLoadRequest;
 import org.apache.impala.thrift.TResetMetadataRequest;
 import org.apache.impala.thrift.TSentryAdminCheckRequest;
+import org.apache.impala.thrift.TSentryAdminCheckResponse;
 import org.apache.impala.thrift.TStatus;
 import org.apache.impala.thrift.TUniqueId;
 import org.apache.impala.thrift.TUpdateCatalogRequest;
@@ -298,20 +299,21 @@ public class JniCatalog {
   }
 
   /**
-   * Verifies whether the user is configured as an admin. Throws an AuthorizationException
-   * if the user does not have admin privileges.
-   *
-   * TODO: rename this method name.
+   * Verifies whether the user is configured as a Sentry admin.
    */
-  public void checkUserSentryAdmin(byte[] thriftReq) throws ImpalaException,
-      TException  {
+  public byte[] checkUserSentryAdmin(byte[] thriftReq)
+      throws ImpalaException, TException {
     TSentryAdminCheckRequest request = new TSentryAdminCheckRequest();
     JniUtil.deserializeThrift(protocolFactory_, request, thriftReq);
+    TSerializer serializer = new TSerializer(protocolFactory_);
     User user = new User(request.getHeader().getRequesting_user());
-    if (!catalogOpExecutor_.getAuthzManager().isAdmin(user)) {
-      throw new AuthorizationException(String.format("User '%s' does not have " +
-          "privileges to access the requested policy metadata.", user.getName()));
-    }
+    Preconditions.checkState(catalogOpExecutor_.getAuthzManager() instanceof
+        SentryCatalogdAuthorizationManager);
+
+    TSentryAdminCheckResponse response = new TSentryAdminCheckResponse();
+    response.setIs_admin(((SentryCatalogdAuthorizationManager)
+        catalogOpExecutor_.getAuthzManager()).isSentryAdmin(user));
+    return serializer.serialize(response);
   }
 
   /**
diff --git a/tests/authorization/test_sentry.py b/tests/authorization/test_sentry.py
new file mode 100644
index 0000000..494fdb5
--- /dev/null
+++ b/tests/authorization/test_sentry.py
@@ -0,0 +1,60 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import pytest
+import os
+from getpass import getuser
+
+from tests.common.custom_cluster_test_suite import CustomClusterTestSuite
+
+SENTRY_CONFIG_DIR = os.getenv('IMPALA_HOME') + '/fe/src/test/resources/'
+SENTRY_CONFIG_FILE = SENTRY_CONFIG_DIR + 'sentry-site.xml'
+
+
+class TestSentry(CustomClusterTestSuite):
+  """This class contains Sentry specific authorization tests."""
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+      impalad_args="--server_name=server1 --sentry_config={0}".format(SENTRY_CONFIG_FILE),
+      catalogd_args="--sentry_config={0}".format(SENTRY_CONFIG_FILE))
+  def test_sentry_admin(self, unique_role, unique_name):
+    """
+    Tests that only admin user can execute certain authorization statements.
+    """
+    admin = getuser()
+    non_admin = unique_name
+    admin_client = self.create_impala_client()
+    non_admin_client = self.create_impala_client()
+    try:
+      self.execute_query_expect_success(admin_client,
+                                        "create role {0}".format(unique_role),
+                                        user=admin)
+
+      # show current roles is always allowed.
+      self.execute_query_expect_success(admin_client, "show current roles", user=admin)
+      self.execute_query_expect_success(non_admin_client, "show current roles",
+                                        user=non_admin)
+
+      for statement in ["show roles",
+                        "show grant role {0}".format(unique_role)]:
+        self.execute_query_expect_success(admin_client, statement, user=admin)
+        result = self.execute_query_expect_failure(non_admin_client, statement,
+                                                   user=non_admin)
+        assert "AuthorizationException: User '{0}' does not have privileges to access " \
+               "the requested policy metadata.".format(non_admin) in str(result)
+    finally:
+      admin_client.execute("drop role {0}".format(unique_role))