You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by as...@apache.org on 2019/12/20 00:55:37 UTC

[impala] branch master updated (4d9d6b4 -> ed5e7da)

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

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


    from 4d9d6b4  Fix bug in report_benchmark_results.py
     new e081e42  IMPALA-9231: support customized privilege checks for SHOW visibility
     new ed5e7da  IMPALA-9240: add HTTP code handling to THttpClient.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/common/global-flags.cc                      |  18 ++-
 be/src/util/backend-gflag-util.cc                  |   4 +-
 common/thrift/BackendGflags.thrift                 |   2 +-
 .../impala/authorization/AuthorizationChecker.java |  10 +-
 .../authorization/BaseAuthorizationChecker.java    |  18 ++-
 .../authorization/PrivilegeRequestBuilder.java     |  23 ++++
 .../org/apache/impala/service/BackendConfig.java   |   4 +-
 .../java/org/apache/impala/service/Frontend.java   |  45 +++++--
 shell/impala_client.py                             |  16 ++-
 tests/authorization/test_authorization.py          | 145 +++++++++++++++++----
 tests/shell/test_shell_interactive.py              |  49 ++++++-
 11 files changed, 279 insertions(+), 55 deletions(-)


[impala] 02/02: IMPALA-9240: add HTTP code handling to THttpClient.

Posted by as...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit ed5e7dae948c50cef7311358b1e096a08b8d8d01
Author: Andrew Sherman <as...@cloudera.com>
AuthorDate: Wed Dec 11 09:38:27 2019 -0800

    IMPALA-9240: add HTTP code handling to THttpClient.
    
    Before this change Impala Shell is not checking HTTP return codes when
    using the hs2-http protocol. The shell is sending a request message
    (e.g. send_CloseOperation) but the HTTP call to send this message may
    fail. This will result in a failure when reading the reply (e.g. in
    recv_CloseOperation) as there is no reply data to read. This will
    typically result in an 'EOFError'.
    
    In code that overrides THttpClient.flush(), check the HTTP code that is
    returned after the HTTP call is made. If the code is not 1XX
    (informational response) or 2XX (successful) then throw an RPCException.
    
    This change does not contain any attempt to recover from an HTTP failures
    but it does allow the failure to be detected and a message to be
    printed.
    
    In future it may be possible to retry after certain HTTP errors.
    
    Testing:
    - Add a new test for impala-shell that tries to connect to an HTTP
      server that always returns a 503 error. Check that an appropriate
      error message is printed.
    
    Change-Id: I3c105f4b8237b87695324d759ffff81821c08c43
    Reviewed-on: http://gerrit.cloudera.org:8080/14924
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 shell/impala_client.py                | 16 ++++++++++--
 tests/shell/test_shell_interactive.py | 49 ++++++++++++++++++++++++++++++++++-
 2 files changed, 62 insertions(+), 3 deletions(-)

diff --git a/shell/impala_client.py b/shell/impala_client.py
index 66eed04..aaccf61 100755
--- a/shell/impala_client.py
+++ b/shell/impala_client.py
@@ -129,6 +129,17 @@ class MissingThriftMethodException(Exception):
       return self.value
 
 
+class CodeCheckingHttpClient(THttpClient):
+  """Add HTTP response code handling to THttpClient."""
+  def flush(self):
+    THttpClient.flush(self)
+    # At this point the http call has completed.
+    if self.code >= 300:
+      # Report any http response code that is not 1XX (informational response) or
+      # 2XX (successful).
+      raise RPCException("HTTP code {}: {}".format(self.code, self.message))
+
+
 def print_to_stderr(message):
   print >> sys.stderr, message
 
@@ -394,10 +405,11 @@ class ImpalaClient(object):
       else:
         ssl_ctx.check_hostname = False  # Mandated by the SSL lib for CERT_NONE mode.
         ssl_ctx.verify_mode = ssl.CERT_NONE
-      transport = THttpClient(
+      transport = CodeCheckingHttpClient(
           "https://{0}/{1}".format(host_and_port, self.http_path), ssl_context=ssl_ctx)
     else:
-      transport = THttpClient("http://{0}/{1}".format(host_and_port, self.http_path))
+      transport = CodeCheckingHttpClient("http://{0}/{1}".
+          format(host_and_port, self.http_path))
 
     if self.use_ldap:
       # Set the BASIC auth header
diff --git a/tests/shell/test_shell_interactive.py b/tests/shell/test_shell_interactive.py
index d95180e..a68b2ef 100755
--- a/tests/shell/test_shell_interactive.py
+++ b/tests/shell/test_shell_interactive.py
@@ -18,6 +18,7 @@
 # specific language governing permissions and limitations
 # under the License.
 
+import httplib
 import logging
 import os
 import pexpect
@@ -26,7 +27,9 @@ import re
 import signal
 import socket
 import sys
+import threading
 from time import sleep
+from contextlib import closing
 
 # This import is the actual ImpalaShell class from impala_shell.py.
 # We rename it to ImpalaShellClass here because we later import another
@@ -40,7 +43,9 @@ from tests.common.impala_test_suite import ImpalaTestSuite
 from tests.common.skip import SkipIfLocal
 from tests.common.test_dimensions import create_client_protocol_dimension
 from util import (assert_var_substitution, ImpalaShell, get_impalad_port, get_shell_cmd,
-                  get_open_sessions_metric)
+                  get_open_sessions_metric, IMPALA_SHELL_EXECUTABLE)
+import SimpleHTTPServer
+import SocketServer
 
 QUERY_FILE_PATH = os.path.join(os.environ['IMPALA_HOME'], 'tests', 'shell')
 
@@ -70,6 +75,20 @@ def tmp_history_file(request):
   return tmp.name
 
 
+class UnavailableRequestHandler(SimpleHTTPServer.SimpleHTTPRequestHandler):
+  """An HTTP server that always returns 503"""
+  def do_POST(self):
+    self.send_response(code=httplib.SERVICE_UNAVAILABLE, message="Service Unavailable")
+
+
+def get_unused_port():
+  """ Find an unused port http://stackoverflow.com/questions/1365265 """
+  with closing(socket.socket(socket.AF_INET, socket.SOCK_STREAM)) as s:
+    s.bind(('', 0))
+    s.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR, 1)
+    return s.getsockname()[1]
+
+
 class TestImpalaShellInteractive(ImpalaTestSuite):
   """Test the impala shell interactively"""
 
@@ -842,6 +861,34 @@ class TestImpalaShellInteractive(ImpalaTestSuite):
       result = p.get_result()
       assert "Fetched 0 row" in result.stderr
 
+  def test_http_codes(self, vector):
+    """Check that the shell prints a good message when using hs2-http protocol
+    and the http server returns a 503 error."""
+    protocol = vector.get_value("protocol")
+    if protocol != 'hs2-http':
+      pytest.skip()
+
+    # Start an http server that always returns 503.
+    HOST = "localhost"
+    PORT = get_unused_port()
+    httpd = None
+    http_server_thread = None
+    try:
+      httpd = SocketServer.TCPServer((HOST, PORT), UnavailableRequestHandler)
+      http_server_thread = threading.Thread(target=httpd.serve_forever)
+      http_server_thread.start()
+
+      # Check that we get a message about the 503 error when we try to connect.
+      shell_args = ["--protocol={0}".format(protocol), "-i{0}:{1}".format(HOST, PORT)]
+      shell_proc = pexpect.spawn(IMPALA_SHELL_EXECUTABLE, shell_args)
+      shell_proc.expect("HTTP code 503", timeout=10)
+    finally:
+      # Clean up.
+      if httpd is not None:
+        httpd.shutdown()
+      if http_server_thread is not None:
+        http_server_thread.join()
+
 
 def run_impala_shell_interactive(vector, input_lines, shell_args=None,
                                  wait_until_connected=True):


[impala] 01/02: IMPALA-9231: support customized privilege checks for SHOW visibility

Posted by as...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e081e42f684593f176e1f1989db26598d79aebbe
Author: stiga-huang <hu...@gmail.com>
AuthorDate: Fri Dec 13 20:56:02 2019 +0800

    IMPALA-9231: support customized privilege checks for SHOW visibility
    
    In IMPALA-9002 we introduce a flag simplify_check_on_show_tables which
    simplifies privilege checks for SHOW TABLES. Only tables with privileges
    implying SELECT privilege will be shown.
    
    This patch provides the same mechanism for SHOW DATABASES. Also augment
    the flag to be a list of privilege names and rename it to
    min_privilege_set_for_show_stmts. The default value is "any" which
    remains the default behavior. If set to "select", only dbs/tables on
    which the user has SELECT privilege will be shown. If set to
    "select,insert", only dbs/tables on which the user has SELECT or INSERT
    privilege will be shown.
    
    Tests:
     - Add tests in test_authorization.py
    
    Change-Id: I631fc5c386a52f0a1f62182473be15fcc3dd8609
    Reviewed-on: http://gerrit.cloudera.org:8080/14904
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/common/global-flags.cc                      |  18 ++-
 be/src/util/backend-gflag-util.cc                  |   4 +-
 common/thrift/BackendGflags.thrift                 |   2 +-
 .../impala/authorization/AuthorizationChecker.java |  10 +-
 .../authorization/BaseAuthorizationChecker.java    |  18 ++-
 .../authorization/PrivilegeRequestBuilder.java     |  23 ++++
 .../org/apache/impala/service/BackendConfig.java   |   4 +-
 .../java/org/apache/impala/service/Frontend.java   |  45 +++++--
 tests/authorization/test_authorization.py          | 145 +++++++++++++++++----
 9 files changed, 217 insertions(+), 52 deletions(-)

diff --git a/be/src/common/global-flags.cc b/be/src/common/global-flags.cc
index c2a4803..451da40 100644
--- a/be/src/common/global-flags.cc
+++ b/be/src/common/global-flags.cc
@@ -279,12 +279,18 @@ DEFINE_bool_hidden(recursively_list_partitions, true,
 DEFINE_bool(unlock_zorder_sort, false,
     "(Experimental) If true, enables using ZORDER option for SORT BY.");
 
-DEFINE_bool(simplify_check_on_show_tables, false,
-    "If true, only check SELECT privilege on SHOW TABLES or GET_TABLES when enabling "
-    "authorization. If false, all privileges will be checked for visibility of a table. "
-    "A table will show up if the user has any privileges on it. This flag is used to "
-    "improve SHOW TABLES performance when using Sentry and have thousands of candidate "
-    "tables to be checked. No performance gain is found in using Ranger");
+DEFINE_string(min_privilege_set_for_show_stmts, "any",
+    "Comma separated list of privileges. Any one of them is required to show a database "
+    "or table. Defaults to \"any\" which means if the user has any privilege (CREATE, "
+    "SELECT, INSERT, etc) on a database or table, the database/table is visible in the "
+    "results of SHOW DATABASES/TABLES. If set to \"select\", only dbs/tables on which "
+    "the user has SELECT privilege will be shown. If set to \"select,insert\", only "
+    "dbs/tables on which the user has SELECT or INSERT privilege will be shown. In "
+    "practice, this flag can be set to \"select\" or \"select,insert\" to improve "
+    "performance of SHOW DATABASES/TABLES and GET_SCHEMAS/GET_TABLES, especially when "
+    "using Sentry and having thousands of candidate dbs/tables to be checked with a "
+    "user with large scale of privileges. No significant performance gain when using "
+    "Ranger");
 
 // Set the slow RPC threshold to 2 minutes to avoid false positives (since TransmitData
 // RPCs can take some time to process).
diff --git a/be/src/util/backend-gflag-util.cc b/be/src/util/backend-gflag-util.cc
index c5eb39e..6906e1d 100644
--- a/be/src/util/backend-gflag-util.cc
+++ b/be/src/util/backend-gflag-util.cc
@@ -81,7 +81,7 @@ DECLARE_bool(use_dedicated_coordinator_estimates);
 DECLARE_string(blacklisted_dbs);
 DECLARE_bool(unlock_zorder_sort);
 DECLARE_string(blacklisted_tables);
-DECLARE_bool(simplify_check_on_show_tables);
+DECLARE_string(min_privilege_set_for_show_stmts);
 DECLARE_int32(num_expected_executors);
 
 namespace impala {
@@ -167,7 +167,7 @@ Status GetThriftBackendGflags(JNIEnv* jni_env, jbyteArray* cfg_bytes) {
   cfg.__set_blacklisted_dbs(FLAGS_blacklisted_dbs);
   cfg.__set_unlock_zorder_sort(FLAGS_unlock_zorder_sort);
   cfg.__set_blacklisted_tables(FLAGS_blacklisted_tables);
-  cfg.__set_simplify_check_on_show_tables(FLAGS_simplify_check_on_show_tables);
+  cfg.__set_min_privilege_set_for_show_stmts(FLAGS_min_privilege_set_for_show_stmts);
   cfg.__set_num_expected_executors(FLAGS_num_expected_executors);
   RETURN_IF_ERROR(SerializeThriftMsg(jni_env, &cfg, cfg_bytes));
   return Status::OK();
diff --git a/common/thrift/BackendGflags.thrift b/common/thrift/BackendGflags.thrift
index 7fc47d5..3489793 100644
--- a/common/thrift/BackendGflags.thrift
+++ b/common/thrift/BackendGflags.thrift
@@ -142,7 +142,7 @@ struct TBackendGflags {
 
   59: required bool unlock_zorder_sort
 
-  60: required bool simplify_check_on_show_tables
+  60: required string min_privilege_set_for_show_stmts
 
   61: required bool mt_dop_auto_fallback
 
diff --git a/fe/src/main/java/org/apache/impala/authorization/AuthorizationChecker.java b/fe/src/main/java/org/apache/impala/authorization/AuthorizationChecker.java
index 8038913..e590d91 100644
--- a/fe/src/main/java/org/apache/impala/authorization/AuthorizationChecker.java
+++ b/fe/src/main/java/org/apache/impala/authorization/AuthorizationChecker.java
@@ -30,7 +30,7 @@ import java.util.Set;
  * An interface used to check whether a user has access to a given resource.
  */
 public interface AuthorizationChecker {
-  /*
+  /**
    * Returns true if the given user has permission to execute the given
    * request, false otherwise. Always returns true if authorization is disabled or the
    * given user is an admin user.
@@ -38,6 +38,14 @@ public interface AuthorizationChecker {
   boolean hasAccess(User user, PrivilegeRequest request) throws InternalException;
 
   /**
+   * Returns true if the given user has permission to execute any of the given
+   * requests, false otherwise. Always returns true if authorization is disabled or the
+   * given user is an admin user.
+   */
+  boolean hasAnyAccess(User user, Set<PrivilegeRequest> requests)
+      throws InternalException;
+
+  /**
    * Creates a a new {@link AuthorizationContext}. {@link AuthorizationContext} gets
    * created per authorization execution.
    *
diff --git a/fe/src/main/java/org/apache/impala/authorization/BaseAuthorizationChecker.java b/fe/src/main/java/org/apache/impala/authorization/BaseAuthorizationChecker.java
index 9e171d9..3920f83 100644
--- a/fe/src/main/java/org/apache/impala/authorization/BaseAuthorizationChecker.java
+++ b/fe/src/main/java/org/apache/impala/authorization/BaseAuthorizationChecker.java
@@ -47,7 +47,7 @@ public abstract class BaseAuthorizationChecker implements AuthorizationChecker {
 
   protected final AuthorizationConfig config_;
 
-  /*
+  /**
    * Creates a new AuthorizationChecker based on the config values.
    */
   protected BaseAuthorizationChecker(AuthorizationConfig config) {
@@ -55,7 +55,7 @@ public abstract class BaseAuthorizationChecker implements AuthorizationChecker {
     config_ = config;
   }
 
-  /*
+  /**
    * Returns true if the given user has permission to execute the given
    * request, false otherwise. Always returns true if authorization is disabled or the
    * given user is an admin user.
@@ -83,6 +83,20 @@ public abstract class BaseAuthorizationChecker implements AuthorizationChecker {
     return authorizeResource(authzCtx, user, request);
   }
 
+  @Override
+  public boolean hasAnyAccess(User user, Set<PrivilegeRequest> requests)
+      throws InternalException {
+    Preconditions.checkNotNull(user);
+    Preconditions.checkNotNull(requests);
+
+    for (PrivilegeRequest request : requests) {
+      if (hasAccess(user, request)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   /**
    * Executes code after the authorization check.
    * Override this method to add custom post-authorization check.
diff --git a/fe/src/main/java/org/apache/impala/authorization/PrivilegeRequestBuilder.java b/fe/src/main/java/org/apache/impala/authorization/PrivilegeRequestBuilder.java
index f87518d..a41b098 100644
--- a/fe/src/main/java/org/apache/impala/authorization/PrivilegeRequestBuilder.java
+++ b/fe/src/main/java/org/apache/impala/authorization/PrivilegeRequestBuilder.java
@@ -17,7 +17,11 @@
 
 package org.apache.impala.authorization;
 
+import java.util.EnumSet;
+import java.util.Set;
+
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Sets;
 import org.apache.impala.catalog.FeDb;
 import org.apache.impala.catalog.FeTable;
 
@@ -33,6 +37,7 @@ public class PrivilegeRequestBuilder {
   private final AuthorizableFactory authzFactory_;
   private Authorizable authorizable_;
   private Privilege privilege_;
+  private EnumSet<Privilege> privilegeSet_;
   private boolean grantOption_ = false;
 
   public PrivilegeRequestBuilder(AuthorizableFactory authzFactory) {
@@ -152,6 +157,14 @@ public class PrivilegeRequestBuilder {
   }
 
   /**
+   * Specifies any of the privileges the user needs to have.
+   */
+  public PrivilegeRequestBuilder anyOf(EnumSet<Privilege> privileges) {
+    privilegeSet_ = privileges;
+    return this;
+  }
+
+  /**
    * Specifies the user needs "ALL" privileges
    */
   public PrivilegeRequestBuilder all() {
@@ -184,4 +197,14 @@ public class PrivilegeRequestBuilder {
     Preconditions.checkNotNull(privilege_);
     return new PrivilegeRequest(authorizable_, privilege_, grantOption_);
   }
+
+  public Set<PrivilegeRequest> buildSet() {
+    Preconditions.checkNotNull(authorizable_);
+    Preconditions.checkNotNull(privilegeSet_);
+    Set<PrivilegeRequest> privileges = Sets.newHashSet();
+    for (Privilege p : privilegeSet_) {
+      privileges.add(new PrivilegeRequest(authorizable_, p, grantOption_));
+    }
+    return privileges;
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/service/BackendConfig.java b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
index cf6b767..47c4660 100644
--- a/fe/src/main/java/org/apache/impala/service/BackendConfig.java
+++ b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
@@ -193,8 +193,8 @@ public class BackendConfig {
     backendCfg_.setUnlock_zorder_sort(zOrdering);
   }
 
-  public boolean simplifyCheckOnShowTables() {
-    return backendCfg_.simplify_check_on_show_tables;
+  public String getMinPrivilegeSetForShowStmts() {
+    return backendCfg_.min_privilege_set_for_show_stmts;
   }
 
   // Inits the auth_to_local configuration in the static KerberosName class.
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 2bf1e98..5c36f37 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -23,6 +23,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -168,6 +169,7 @@ import org.slf4j.LoggerFactory;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicates;
+import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.Uninterruptibles;
@@ -249,6 +251,8 @@ public class Frontend {
   private final FeCatalogManager catalogManager_;
   private final AuthorizationFactory authzFactory_;
   private final AuthorizationManager authzManager_;
+  // Privileges in which the user should have any of them to see a database or table,
+  private final EnumSet<Privilege> minPrivilegeSetForShowStmts_;
   /**
    * Authorization checker. Initialized and periodically loaded by a task
    * running on the {@link #policyReader_} thread.
@@ -296,6 +300,7 @@ public class Frontend {
     catalogManager_.setAuthzChecker(authzChecker_);
     authzManager_ = authzFactory.newAuthorizationManager(catalogManager_,
         authzChecker_::get);
+    minPrivilegeSetForShowStmts_ = getMinPrivilegeSetForShowStmts();
     impaladTableUsageTracker_ = ImpaladTableUsageTracker.createFromConfig(
         BackendConfig.INSTANCE);
     queryHookManager_ = QueryEventHookManager.createFromConfig(BackendConfig.INSTANCE);
@@ -307,6 +312,24 @@ public class Frontend {
     }
   }
 
+  /**
+   * Returns the required privilege set for showing a database or table.
+   */
+  private EnumSet<Privilege> getMinPrivilegeSetForShowStmts() throws InternalException {
+    String configStr = BackendConfig.INSTANCE.getMinPrivilegeSetForShowStmts();
+    if (Strings.isNullOrEmpty(configStr)) return EnumSet.of(Privilege.ANY);
+    EnumSet<Privilege> privileges = EnumSet.noneOf(Privilege.class);
+    for (String pStr : configStr.toUpperCase().split(",")) {
+      try {
+        privileges.add(Privilege.valueOf(pStr.trim()));
+      } catch (IllegalArgumentException e) {
+        LOG.error("Illegal privilege name '{}'", pStr, e);
+        throw new InternalException("Failed to parse privileges: " + configStr, e);
+      }
+    }
+    return privileges.isEmpty() ? EnumSet.of(Privilege.ANY) : privileges;
+  }
+
   public FeCatalog getCatalog() { return catalogManager_.getOrCreateCatalog(); }
 
   public AuthorizationChecker getAuthzChecker() { return authzChecker_.get(); }
@@ -774,10 +797,6 @@ public class Frontend {
     FeCatalog catalog = getCatalog();
     List<String> tblNames = catalog.getTableNames(dbName, matcher);
     if (authzFactory_.getAuthorizationConfig().isEnabled()) {
-      Privilege requiredPrivilege = Privilege.ANY;
-      if (BackendConfig.INSTANCE.simplifyCheckOnShowTables()) {
-        requiredPrivilege = Privilege.SELECT;
-      }
       Iterator<String> iter = tblNames.iterator();
       while (iter.hasNext()) {
         String tblName = iter.next();
@@ -794,10 +813,12 @@ public class Frontend {
           LOG.info("Table {} not yet loaded, ignoring it in table listing.",
               dbName + "." + tblName);
         }
-        PrivilegeRequest privilegeRequest = new PrivilegeRequestBuilder(
+        Set<PrivilegeRequest> requests = new PrivilegeRequestBuilder(
             authzFactory_.getAuthorizableFactory())
-            .allOf(requiredPrivilege).onAnyColumn(dbName, tblName, tableOwner).build();
-        if (!authzChecker_.get().hasAccess(user, privilegeRequest)) {
+            .anyOf(minPrivilegeSetForShowStmts_)
+            .onAnyColumn(dbName, tblName, tableOwner)
+            .buildSet();
+        if (!authzChecker_.get().hasAnyAccess(user, requests)) {
           iter.remove();
         }
       }
@@ -940,10 +961,12 @@ public class Frontend {
       // Default DB should always be shown.
       return true;
     }
-    PrivilegeRequest request = new PrivilegeRequestBuilder(
-        authzFactory_.getAuthorizableFactory()).any().onAnyColumn(
-            db.getName(), db.getOwnerUser()).build();
-    return authzChecker_.get().hasAccess(user, request);
+    Set<PrivilegeRequest> requests = new PrivilegeRequestBuilder(
+        authzFactory_.getAuthorizableFactory())
+        .anyOf(minPrivilegeSetForShowStmts_)
+        .onAnyColumn(db.getName(), db.getOwnerUser())
+        .buildSet();
+    return authzChecker_.get().hasAnyAccess(user, requests);
   }
 
   /**
diff --git a/tests/authorization/test_authorization.py b/tests/authorization/test_authorization.py
index d58dea2..885d184 100644
--- a/tests/authorization/test_authorization.py
+++ b/tests/authorization/test_authorization.py
@@ -486,16 +486,21 @@ class TestAuthorization(CustomClusterTestSuite):
       self.execute_query_expect_success(client,
                                         "select * from functional.alltypes limit 1")
 
-  @pytest.mark.execute_serially
-  @CustomClusterTestSuite.with_args(
-    impalad_args="--server_name=server1 --sentry_config=%s "
-                 "--authorized_proxy_user_config=%s=* "
-                 "--simplify_check_on_show_tables=true" %
-                 (SENTRY_CONFIG_FILE, getuser()),
-    catalogd_args="--sentry_config={0}".format(SENTRY_CONFIG_FILE),
-    sentry_config=SENTRY_CONFIG_FILE_OO,  # Enable Sentry Object Ownership
-    sentry_log_dir="{0}/test_fast_show_tables_with_sentry".format(SENTRY_BASE_LOG_DIR))
-  def test_fast_show_tables_with_sentry(self, unique_role, unique_name):
+  @staticmethod
+  def _verify_show_dbs(result, unique_name, visibility_privileges=PRIVILEGES):
+    """ Helper function for verifying the results of SHOW DATABASES below.
+    Only show databases with privileges implying any of the visibility_privileges.
+    """
+    for priv in PRIVILEGES:
+      # Result lines are in the format of "db_name\tdb_comment"
+      db_name = 'db_%s_%s\t' % (unique_name, priv)
+      if priv != 'all' and priv not in visibility_privileges:
+        assert db_name not in result.data
+      else:
+        assert db_name in result.data
+
+  def _test_sentry_show_stmts_helper(self, unique_role, unique_name,
+                                     visibility_privileges):
     unique_db = unique_name + "_db"
     # TODO: can we create and use a temp username instead of using root?
     another_user = 'root'
@@ -511,52 +516,138 @@ class TestAuthorization(CustomClusterTestSuite):
       self.client.execute("drop database if exists %s cascade" % unique_db)
       self.client.execute("create database %s" % unique_db)
       for priv in PRIVILEGES:
+        self.client.execute("create database db_%s_%s" % (unique_name, priv))
+        self.client.execute("grant {0} on database db_{1}_{2} to role {3}"
+                            .format(priv, unique_name, priv, unique_role))
         self.client.execute("create table %s.tbl_%s (i int)" % (unique_db, priv))
         self.client.execute("grant {0} on table {1}.tbl_{2} to role {3}"
                             .format(priv, unique_db, priv, unique_role))
       self.client.execute("grant role %s to group %s" %
                           (unique_role, another_user_grp))
 
-      # Owner (current user) can still see all the tables
+      # Owner (current user) can still see all the owned databases and tables
+      result = self.client.execute("show databases")
+      TestAuthorization._verify_show_dbs(result, unique_name)
       result = self.client.execute("show tables in %s" % unique_db)
       assert result.data == ["tbl_%s" % p for p in PRIVILEGES]
 
-      # Check SHOW TABLES using another username
+      # Check SHOW DATABASES and SHOW TABLES using another username
       # Create another client so we can user another username
       root_impalad_client = self.create_impala_client()
       result = self.execute_query_expect_success(
-        root_impalad_client, "show tables in %s" % unique_db, user=another_user)
-      # Only show tables with privileges implying SELECT privilege
-      assert result.data == ['tbl_all', 'tbl_select']
+          root_impalad_client, "show databases", user=another_user)
+      TestAuthorization._verify_show_dbs(result, unique_name, visibility_privileges)
+      result = self.execute_query_expect_success(
+          root_impalad_client, "show tables in %s" % unique_db, user=another_user)
+      # Only show tables with privileges implying any of the visibility privileges
+      assert 'tbl_all' in result.data   # ALL can imply to any privilege
+      for p in visibility_privileges:
+        assert 'tbl_%s' % p in result.data
     finally:
-      self.role_cleanup(unique_role)
       self.client.execute("drop database if exists %s cascade" % unique_db)
+      for priv in PRIVILEGES:
+        self.client.execute(
+            "drop database if exists db_%s_%s cascade" % (unique_name, priv))
+      self.role_cleanup(unique_role)
 
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(
-    impalad_args="--server-name=server1 --ranger_service_type=hive "
-                 "--ranger_app_id=impala --authorization_provider=ranger "
-                 "--simplify_check_on_show_tables=true",
-    catalogd_args="--server-name=server1 --ranger_service_type=hive "
-                  "--ranger_app_id=impala --authorization_provider=ranger")
-  def test_fast_show_tables_with_ranger(self, unique_role, unique_name):
+    impalad_args="--server_name=server1 --sentry_config=%s "
+                 "--authorized_proxy_user_config=%s=* "
+                 "--min_privilege_set_for_show_stmts=select" %
+                 (SENTRY_CONFIG_FILE, getuser()),
+    catalogd_args="--sentry_config={0}".format(SENTRY_CONFIG_FILE),
+    sentry_config=SENTRY_CONFIG_FILE_OO,  # Enable Sentry Object Ownership
+    sentry_log_dir="{0}/test_sentry_show_stmts_with_select".format(SENTRY_BASE_LOG_DIR))
+  def test_sentry_show_stmts_with_select(self, unique_role, unique_name):
+    self._test_sentry_show_stmts_helper(unique_role, unique_name, ['select'])
+
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+    impalad_args="--server_name=server1 --sentry_config=%s "
+                 "--authorized_proxy_user_config=%s=* "
+                 "--min_privilege_set_for_show_stmts=select,insert" %
+                 (SENTRY_CONFIG_FILE, getuser()),
+    catalogd_args="--sentry_config={0}".format(SENTRY_CONFIG_FILE),
+    sentry_config=SENTRY_CONFIG_FILE_OO,  # Enable Sentry Object Ownership
+    sentry_log_dir="{0}/test_sentry_show_stmts_with_select_insert"
+                   .format(SENTRY_BASE_LOG_DIR))
+  def test_sentry_show_stmts_with_select_insert(self, unique_role, unique_name):
+    self._test_sentry_show_stmts_helper(unique_role, unique_name,
+                                        ['select', 'insert'])
+
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+    impalad_args="--server_name=server1 --sentry_config=%s "
+                 "--authorized_proxy_user_config=%s=* "
+                 "--min_privilege_set_for_show_stmts=any" %
+                 (SENTRY_CONFIG_FILE, getuser()),
+    catalogd_args="--sentry_config={0}".format(SENTRY_CONFIG_FILE),
+    sentry_config=SENTRY_CONFIG_FILE_OO,  # Enable Sentry Object Ownership
+    sentry_log_dir="{0}/test_sentry_show_stmts_with_any".format(SENTRY_BASE_LOG_DIR))
+  def test_sentry_show_stmts_with_any(self, unique_role, unique_name):
+    self._test_sentry_show_stmts_helper(unique_role, unique_name, PRIVILEGES)
+
+  def _test_ranger_show_stmts_helper(self, unique_name, visibility_privileges):
     unique_db = unique_name + "_db"
     admin_client = self.create_impala_client()
     try:
       admin_client.execute("drop database if exists %s cascade" % unique_db, user=ADMIN)
       admin_client.execute("create database %s" % unique_db, user=ADMIN)
       for priv in PRIVILEGES:
+        admin_client.execute("create database db_%s_%s" % (unique_name, priv))
+        admin_client.execute("grant {0} on database db_{1}_{2} to user {3}"
+                             .format(priv, unique_name, priv, getuser()))
         admin_client.execute("create table %s.tbl_%s (i int)" % (unique_db, priv))
         admin_client.execute("grant {0} on table {1}.tbl_{2} to user {3}"
-                            .format(priv, unique_db, priv, getuser()))
+                             .format(priv, unique_db, priv, getuser()))
 
-      # Admin can still see all the tables
+      # Admin can still see all the databases and tables
+      result = admin_client.execute("show databases")
+      TestAuthorization._verify_show_dbs(result, unique_name)
       result = admin_client.execute("show tables in %s" % unique_db)
       assert result.data == ["tbl_%s" % p for p in PRIVILEGES]
 
-      # Check SHOW TABLES using another username
+      # Check SHOW DATABASES and SHOW TABLES using another username
+      result = self.client.execute("show databases")
+      TestAuthorization._verify_show_dbs(result, unique_name, visibility_privileges)
       result = self.client.execute("show tables in %s" % unique_db)
-      # Only show tables with privileges implying SELECT privilege
-      assert result.data == ['tbl_all', 'tbl_select']
+      # Only show tables with privileges implying any of the visibility privileges
+      assert 'tbl_all' in result.data   # ALL can imply to any privilege
+      for p in visibility_privileges:
+        assert 'tbl_%s' % p in result.data
     finally:
       admin_client.execute("drop database if exists %s cascade" % unique_db)
+      for priv in PRIVILEGES:
+        admin_client.execute(
+            "drop database if exists db_%s_%s cascade" % (unique_name, priv))
+
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+    impalad_args="--server-name=server1 --ranger_service_type=hive "
+                 "--ranger_app_id=impala --authorization_provider=ranger "
+                 "--min_privilege_set_for_show_stmts=select",
+    catalogd_args="--server-name=server1 --ranger_service_type=hive "
+                  "--ranger_app_id=impala --authorization_provider=ranger")
+  def test_ranger_show_stmts_with_select(self, unique_name):
+    self._test_ranger_show_stmts_helper(unique_name, ['select'])
+
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+    impalad_args="--server-name=server1 --ranger_service_type=hive "
+                 "--ranger_app_id=impala --authorization_provider=ranger "
+                 "--min_privilege_set_for_show_stmts=select,insert",
+    catalogd_args="--server-name=server1 --ranger_service_type=hive "
+                  "--ranger_app_id=impala --authorization_provider=ranger")
+  def test_ranger_show_stmts_with_select_insert(self, unique_name):
+    self._test_ranger_show_stmts_helper(unique_name, ['select', 'insert'])
+
+  @pytest.mark.execute_serially
+  @CustomClusterTestSuite.with_args(
+    impalad_args="--server-name=server1 --ranger_service_type=hive "
+                 "--ranger_app_id=impala --authorization_provider=ranger "
+                 "--min_privilege_set_for_show_stmts=any",
+    catalogd_args="--server-name=server1 --ranger_service_type=hive "
+                  "--ranger_app_id=impala --authorization_provider=ranger")
+  def test_ranger_show_stmts_with_any(self, unique_name):
+    self._test_ranger_show_stmts_helper(unique_name, PRIVILEGES)