You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by gr...@apache.org on 2019/06/10 20:58:49 UTC

[kudu] branch master updated (71b0940 -> ec1a1f3)

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

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


    from 71b0940  [hms] Adjust storage handler package
     new 3c4635f  authz: refactor authorization for ListTables
     new d3820b3  sentry: allow caching of COLUMN/TABLE privileges when checking higher scopes
     new ec1a1f3  thirdparty: fix build_curl with unusual krb5-config location

The 3 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:
 src/kudu/integration-tests/master_sentry-itest.cc | 25 ++++++
 src/kudu/master/authz_provider.h                  | 11 +++
 src/kudu/master/catalog_manager.cc                | 55 ++++++++++---
 src/kudu/master/catalog_manager.h                 |  5 +-
 src/kudu/master/default_authz_provider.h          |  9 +++
 src/kudu/master/sentry_authz_provider-test.cc     | 12 +--
 src/kudu/master/sentry_authz_provider.cc          | 48 +++++++++---
 src/kudu/master/sentry_authz_provider.h           | 19 ++++-
 src/kudu/master/sentry_privileges_fetcher.cc      | 96 ++++++++++++-----------
 src/kudu/master/sentry_privileges_fetcher.h       | 14 +++-
 thirdparty/build-definitions.sh                   | 11 +++
 11 files changed, 226 insertions(+), 79 deletions(-)


[kudu] 01/03: authz: refactor authorization for ListTables

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

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

commit 3c4635fc51da932b8a6f8c779a1c4f3c09e5b7ca
Author: Andrew Wong <aw...@apache.org>
AuthorDate: Mon Jun 3 19:43:11 2019 -0400

    authz: refactor authorization for ListTables
    
    Authorization for ListTables isn't very performant because it requires
    authorizing each table while holding its table lock. This invariant is
    held to guarantee that the authorization applies to the correct table in
    Kudu (e.g. even amidst a concurrent rename).
    
    If we authorize multiple tables at once, we can improve performance, but
    keeping the guarantee is less straightforward. This patch takes a stab
    at this by refactoring ListTables to prepare for the following
    authorization sequence:
    
    1. With table locks held, put together a map from
       { table_name => TableInfo }.
    2. Authorize the tables (potentially in bulk in the future), keeping
       track of the authorized table names.
    3. Iterate through the authorized table names and, using the maps from
       Step 1, with table locks held, check that the table name hasn't
       changed. If it has, this implies there was a concurrent rename (or
       equivalent) and the authorization call was not actually made for this
       table, and it shouldn't be returned.
    4. With confirmation that the tables that were authorized have the IDs
       we expect, return the authorized tables.
    
    If the catalog manager isn't set up to authorize tables (e.g.
    authorization isn't enabled), steps 2 and 3 are skipped. This patch
    refactors ListTables to match this sequence of events without bulk
    authorization.
    
    While this is mainly a refactor, the user-facing change is that if there
    is a concurrent rename during a ListTables operation, there is a chance
    that neither the old nor new table will show up in the listed tables.
    This isn't ideal UX, but is conservative w.r.t security. A test is added
    exercising this scenario.
    
    Change-Id: I568e1be7b909768a99f0f4f13775e34a01ccd160
    Reviewed-on: http://gerrit.cloudera.org:8080/13566
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Tested-by: Kudu Jenkins
    Reviewed-by: Hao Hao <ha...@cloudera.com>
---
 src/kudu/integration-tests/master_sentry-itest.cc | 25 +++++++++++
 src/kudu/master/authz_provider.h                  | 11 +++++
 src/kudu/master/catalog_manager.cc                | 55 +++++++++++++++++------
 src/kudu/master/catalog_manager.h                 |  5 +--
 src/kudu/master/default_authz_provider.h          |  9 ++++
 src/kudu/master/sentry_authz_provider.cc          | 15 +++++++
 src/kudu/master/sentry_authz_provider.h           |  5 +++
 7 files changed, 109 insertions(+), 16 deletions(-)

diff --git a/src/kudu/integration-tests/master_sentry-itest.cc b/src/kudu/integration-tests/master_sentry-itest.cc
index ddaf02d..8f7f849 100644
--- a/src/kudu/integration-tests/master_sentry-itest.cc
+++ b/src/kudu/integration-tests/master_sentry-itest.cc
@@ -450,6 +450,31 @@ TEST_F(MasterSentryTest, TestAuthzListTables) {
   ASSERT_EQ(unordered_set<string>({ table_name, sec_table_name }), tables_set);
 }
 
+// When authorizing ListTables, if there is a concurrent rename, we may not end
+// up showing the table.
+TEST_F(MasterSentryTest, TestAuthzListTablesConcurrentRename) {
+  ASSERT_OK(cluster_->SetFlag(cluster_->master(),
+      "catalog_manager_inject_latency_list_authz_ms", "3000"));;
+  const auto table_name = Substitute("$0.$1", kDatabaseName, kTableName);
+  const auto sec_table_name = Substitute("$0.$1", kDatabaseName, kSecondTable);
+  ASSERT_OK(GrantGetMetadataTablePrivilege({ kDatabaseName, kTableName }));
+  ASSERT_OK(GrantRenameTablePrivilege({ kDatabaseName, kTableName }));
+
+  // List the tables while injecting latency.
+  vector<string> tables;
+  thread t([&] {
+    ASSERT_OK(client_->ListTables(&tables));
+  });
+
+  // While that's happening, rename one of the tables.
+  ASSERT_OK(RenameTable({ table_name, Substitute("$0.$1", kDatabaseName, "b") }));
+  NO_FATALS(t.join());
+
+  // We shouldn't see the renamed table.
+  ASSERT_EQ(1, tables.size());
+  ASSERT_EQ(sec_table_name, tables[0]);
+}
+
 TEST_F(MasterSentryTest, TestTableOwnership) {
   ASSERT_OK(GrantCreateTablePrivilege({ kDatabaseName }));
   ASSERT_OK(CreateKuduTable(kDatabaseName, "new_table"));
diff --git a/src/kudu/master/authz_provider.h b/src/kudu/master/authz_provider.h
index d768518..51ba635 100644
--- a/src/kudu/master/authz_provider.h
+++ b/src/kudu/master/authz_provider.h
@@ -84,6 +84,17 @@ class AuthzProvider {
   virtual Status AuthorizeGetTableMetadata(const std::string& table_name,
                                            const std::string& user) WARN_UNUSED_RESULT = 0;
 
+  // Filters the given table names, removing any the user is not authorized to
+  // see.
+  //
+  // Sets 'checked_table_names' if the AuthzProvider actually checked
+  // privileges for the table (rather than just passing through). This may be
+  // useful, e.g. to indicate that the caller needs to verify the table names
+  // have not changed during authorization.
+  virtual Status AuthorizeListTables(const std::string& user,
+                                     std::unordered_set<std::string>* table_names,
+                                     bool* checked_table_names) WARN_UNUSED_RESULT = 0;
+
   // Populates the privilege fields of 'pb' with the table-specific privileges
   // for the given user, using 'schema_pb' for metadata (e.g. column IDs). This
   // does not populate the table ID field of 'pb' -- only the privilege fields;
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 8e2ef75..a2c4cd8 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -261,6 +261,12 @@ DEFINE_bool(catalog_manager_evict_excess_replicas, true,
 TAG_FLAG(catalog_manager_evict_excess_replicas, hidden);
 TAG_FLAG(catalog_manager_evict_excess_replicas, runtime);
 
+DEFINE_int32(catalog_manager_inject_latency_list_authz_ms, 0,
+             "Injects a sleep in milliseconds while authorizing a ListTables "
+             "request. This is a test-only flag.");
+TAG_FLAG(catalog_manager_inject_latency_list_authz_ms, hidden);
+TAG_FLAG(catalog_manager_inject_latency_list_authz_ms, unsafe);
+
 DECLARE_bool(raft_prepare_replacement_before_eviction);
 DECLARE_bool(raft_attempt_to_replace_replica_without_majority);
 DECLARE_int64(tsk_rotation_seconds);
@@ -2811,34 +2817,57 @@ Status CatalogManager::ListTables(const ListTablesRequestPB* req,
       tables_info.emplace_back(entry.second);
     }
   }
-
+  unordered_map<string, scoped_refptr<TableInfo>> table_info_by_name;
+  unordered_set<string> table_names;
   for (const auto& table_info : tables_info) {
     TableMetadataLock ltm(table_info.get(), LockMode::READ);
     if (!ltm.data().is_running()) continue; // implies !is_deleted() too
 
-    const string table_name = ltm.data().name();
+    const string& table_name = ltm.data().name();
     if (req->has_name_filter()) {
       size_t found = table_name.find(req->name_filter());
       if (found == string::npos) {
         continue;
       }
     }
+    InsertOrUpdate(&table_info_by_name, table_name, table_info);
+    EmplaceIfNotPresent(&table_names, table_name);
+  }
 
-    // TODO(hao): need to improve the performance before enabling authz by default
-    // for cases there are a lot of tables.
-    if (user) {
-      Status s = authz_provider_->AuthorizeGetTableMetadata(NormalizeTableName(table_name),
-                                                            *user);
-      if (!s.ok()) {
+  MAYBE_INJECT_FIXED_LATENCY(FLAGS_catalog_manager_inject_latency_list_authz_ms);
+  bool checked_table_names = false;
+  if (user) {
+    RETURN_NOT_OK(authz_provider_->AuthorizeListTables(
+        *user, &table_names, &checked_table_names));
+  }
+
+  // If we checked privileges, do another pass over the tables to filter out
+  // any that may have been altered while authorizing.
+  if (checked_table_names) {
+    for (const auto& table_name : table_names) {
+      const auto& table_info = FindOrDie(table_info_by_name, table_name);
+      TableMetadataLock ltm(table_info.get(), LockMode::READ);
+      if (!ltm.data().is_running()) continue;
+
+      // If we have a different table name than expected, there was a table
+      // rename and we shouldn't show the table.
+      if (table_name != ltm.data().name()) {
         continue;
       }
+      ListTablesResponsePB::TableInfo* table = resp->add_tables();
+      table->set_id(table_info->id());
+      table->set_name(table_name);
+    }
+  } else {
+    // Otherwise, pass all tables through.
+    for (const auto& name_and_table_info : table_info_by_name) {
+      const auto& table_name = name_and_table_info.first;
+      const auto& table_info = name_and_table_info.second;
+      ListTablesResponsePB::TableInfo* table = resp->add_tables();
+      table->set_id(table_info->id());
+      table->set_name(table_name);
     }
-
-    ListTablesResponsePB::TableInfo* table = resp->add_tables();
-    table->set_id(table_info->id());
-    table->set_name(ltm.data().name());
   }
-
   return Status::OK();
 }
 
diff --git a/src/kudu/master/catalog_manager.h b/src/kudu/master/catalog_manager.h
index 282e7c3..df20777 100644
--- a/src/kudu/master/catalog_manager.h
+++ b/src/kudu/master/catalog_manager.h
@@ -589,9 +589,8 @@ class CatalogManager : public tserver::TabletReplicaLookupIf {
                         boost::optional<const std::string&> user,
                         const security::TokenSigner* token_signer);
 
-  // List all the running tables. If 'user' is provided, checks that the user
-  // is authorized to get such information, otherwise, only list the tables that
-  // the user has permission on.
+  // Lists all the running tables. If 'user' is provided, only lists those that
+  // the given user is authorized to see.
   Status ListTables(const ListTablesRequestPB* req,
                     ListTablesResponsePB* resp,
                     boost::optional<const std::string&> user);
diff --git a/src/kudu/master/default_authz_provider.h b/src/kudu/master/default_authz_provider.h
index bca2dd6..9db2f5a 100644
--- a/src/kudu/master/default_authz_provider.h
+++ b/src/kudu/master/default_authz_provider.h
@@ -18,9 +18,11 @@
 #pragma once
 
 #include <string>
+#include <unordered_set>
 
 #include <glog/logging.h>
 
+#include "kudu/gutil/port.h"
 #include "kudu/master/authz_provider.h"
 #include "kudu/security/token.pb.h"
 #include "kudu/util/status.h"
@@ -64,6 +66,13 @@ class DefaultAuthzProvider : public AuthzProvider {
     return Status::OK();
   }
 
+  Status AuthorizeListTables(const std::string& /*user*/,
+                             std::unordered_set<std::string>* /*table_names*/,
+                             bool* checked_table_names) override WARN_UNUSED_RESULT {
+    *checked_table_names = false;
+    return Status::OK();
+  }
+
   Status FillTablePrivilegePB(const std::string& /*table_name*/,
                               const std::string& /*user*/,
                               const SchemaPB& /*schema_pb*/,
diff --git a/src/kudu/master/sentry_authz_provider.cc b/src/kudu/master/sentry_authz_provider.cc
index 8dec342..bdc2fdf 100644
--- a/src/kudu/master/sentry_authz_provider.cc
+++ b/src/kudu/master/sentry_authz_provider.cc
@@ -183,6 +183,21 @@ Status SentryAuthzProvider::AuthorizeGetTableMetadata(const string& table_name,
                    table_name, user);
 }
 
+Status SentryAuthzProvider::AuthorizeListTables(const string& user,
+                                                unordered_set<string>* table_names,
+                                                bool* checked_table_names) {
+  unordered_set<string> authorized_tables;
+  for (auto table_name : *table_names) {
+    Status s = AuthorizeGetTableMetadata(table_name, user);
+    if (s.ok()) {
+      EmplaceOrDie(&authorized_tables, std::move(table_name));
+    }
+  }
+  *table_names = authorized_tables;
+  *checked_table_names = true;
+  return Status::OK();
+}
+
 Status SentryAuthzProvider::FillTablePrivilegePB(const string& table_name,
                                                  const string& user,
                                                  const SchemaPB& schema_pb,
diff --git a/src/kudu/master/sentry_authz_provider.h b/src/kudu/master/sentry_authz_provider.h
index aa4a23b..c23e74f 100644
--- a/src/kudu/master/sentry_authz_provider.h
+++ b/src/kudu/master/sentry_authz_provider.h
@@ -18,6 +18,7 @@
 #pragma once
 
 #include <string>
+#include <unordered_set>
 
 #include <gtest/gtest_prod.h>
 
@@ -84,6 +85,10 @@ class SentryAuthzProvider : public AuthzProvider {
   Status AuthorizeGetTableMetadata(const std::string& table_name,
                                    const std::string& user) override WARN_UNUSED_RESULT;
 
+  Status AuthorizeListTables(const std::string& user,
+                             std::unordered_set<std::string>* table_names,
+                             bool* checked_table_names) override WARN_UNUSED_RESULT;
+
   Status FillTablePrivilegePB(const std::string& table_name,
                               const std::string& user,
                               const SchemaPB& schema_pb,


[kudu] 03/03: thirdparty: fix build_curl with unusual krb5-config location

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

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

commit ec1a1f301b6f8b5ca8428d755e7658ec1c3d22a7
Author: Adar Dembo <ad...@cloudera.com>
AuthorDate: Mon Jun 10 12:25:38 2019 -0700

    thirdparty: fix build_curl with unusual krb5-config location
    
    The curl build system runs krb5-config to determine which GSSAPI headers
    and libraries are needed at build time. It expects to find krb5-config in
    /usr/bin and if it's not found, it'll default to linking with -lgssapi.
    
    On SLES12, krb5-config is in /usr/lib/mit/bin, so it isn't found and the
    curl build generally fails because -lgssapi isn't a Kudu requirement. By
    setting KRB5CONFIG we can redirect curl's build towards the correct location
    of krb5-config and get the correct libraries as a result (-lkrb5,
    -lk5crypto, and -lcom_err, if you're curious).
    
    Change-Id: Ief5816d561545945089108e34bd59c8e5dae9dc4
    Reviewed-on: http://gerrit.cloudera.org:8080/13577
    Reviewed-by: Todd Lipcon <to...@apache.org>
    Tested-by: Kudu Jenkins
---
 thirdparty/build-definitions.sh | 11 +++++++++++
 1 file changed, 11 insertions(+)

diff --git a/thirdparty/build-definitions.sh b/thirdparty/build-definitions.sh
index 3aefe40..d156dab 100644
--- a/thirdparty/build-definitions.sh
+++ b/thirdparty/build-definitions.sh
@@ -656,6 +656,16 @@ build_curl() {
   mkdir -p $CURL_BDIR
   pushd $CURL_BDIR
 
+  # curl's configure script expects krb5-config to be in /usr/bin. If that's not
+  # the case (looking at you, SLES12's /usr/lib/mit/bin/krb5-config), we need to
+  # pass the right location via the KRB5CONFIG environment variable.
+  #
+  # TODO(adar): there's gotta be a way to do this without using export/unset.
+  KRB5CONFIG_LOCATION=$(which krb5-config)
+  if [ "$KRB5CONFIG_LOCATION" != "/usr/bin/krb5-config" ]; then
+    export KRB5CONFIG=$KRB5CONFIG_LOCATION
+  fi
+
   # Note: curl shows a message asking for CPPFLAGS to be used for include
   # directories, not CFLAGS.
   CFLAGS="$EXTRA_CFLAGS" \
@@ -681,6 +691,7 @@ build_curl() {
     --without-librtmp \
     --without-libssh2 \
     --with-gssapi
+  unset KRB5CONFIG
   make -j$PARALLEL $EXTRA_MAKEFLAGS install
   popd
 }


[kudu] 02/03: sentry: allow caching of COLUMN/TABLE privileges when checking higher scopes

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

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

commit d3820b3c4458abe768ced6ee683470dc2cc39b50
Author: Andrew Wong <aw...@apache.org>
AuthorDate: Wed Jun 5 16:39:09 2019 -0700

    sentry: allow caching of COLUMN/TABLE privileges when checking higher scopes
    
    Currently, Kudu will not cache COLUMN/TABLE when checking for
    DATABASE/SERVER Sentry privileges. This is because, when written, Kudu
    would send requests that would yield significantly more privileges than
    required. Commit 0040a731e4741bde6d8c9d81e796cb000cd24817 changed this
    behavior, and as such, this patch gates the optimization behind an
    enum in the Sentry provider's Authorize() signature.
    
    The effect of this is that when checking for DATABASE/SERVER privileges,
    two entries will be added to the privilege cache instead of one: one for
    the DATABASE/SERVER privileges, and one for the TABLE/COLUMN privileges.
    
    This is optional because in some cases, it isn't necessarily desirable
    to cache table-level privileges. E.g. when creating a table, we
    shouldn't cache table-level privileges because Sentry may create OWNER
    privileges for the new table that caching may miss out on when first
    authorizing the create request.
    
    This improves the worst case performance of ListTables when there are
    many databases on which the user has no privileges, and a single table
    per database. In this scenario, without this patch, there would be
    2 * (# tables) calls to Sentry, and with this patch, there would be
    1 * (# tables) calls.
    
    With this patch:
    ./bin/sentry_authz_provider-test --gtest_filter=*ListTablesBench* --num_databases=300 --num_tables_per_db=1 --has-db-privileges=false
    sentry_authz_provider-test.cc:418] Time spent Listing tables: real 24.735s  user 0.033s     sys 0.009s
    
    Without this patch:
    ./bin/sentry_authz_provider-test --gtest_filter=*ListTablesBench* --num_databases=300 --num_tables_per_db=1 --has-db-privileges=false
    sentry_authz_provider-test.cc:418] Time spent Listing tables: real 47.543s  user 0.040s     sys 0.013s
    
    Change-Id: Icec75ae9e5626c887af37568a6f64a8361d888b7
    Reviewed-on: http://gerrit.cloudera.org:8080/13552
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/master/sentry_authz_provider-test.cc | 12 ++--
 src/kudu/master/sentry_authz_provider.cc      | 33 ++++++---
 src/kudu/master/sentry_authz_provider.h       | 14 +++-
 src/kudu/master/sentry_privileges_fetcher.cc  | 96 +++++++++++++++------------
 src/kudu/master/sentry_privileges_fetcher.h   | 14 +++-
 5 files changed, 106 insertions(+), 63 deletions(-)

diff --git a/src/kudu/master/sentry_authz_provider-test.cc b/src/kudu/master/sentry_authz_provider-test.cc
index 1b37927..21a8588 100644
--- a/src/kudu/master/sentry_authz_provider-test.cc
+++ b/src/kudu/master/sentry_authz_provider-test.cc
@@ -592,7 +592,8 @@ TEST_P(SentryAuthzProviderFilterPrivilegesScopeTest, TestFilterInvalidResponses)
                                        SentryAuthorizableScope::TABLE }) {
     SentryPrivilegesBranch privileges_info;
     ASSERT_OK(sentry_authz_provider_->fetcher_.GetSentryPrivileges(
-        requested_scope, table_ident, kTestUser, &privileges_info));
+        requested_scope, table_ident, kTestUser,
+        SentryCaching::ALL, &privileges_info));
     // Kudu should ignore all of the invalid privileges.
     ASSERT_TRUE(privileges_info.privileges().empty());
   }
@@ -614,7 +615,8 @@ TEST_P(SentryAuthzProviderFilterPrivilegesScopeTest, TestFilterValidResponses) {
                                        SentryAuthorizableScope::TABLE }) {
     SentryPrivilegesBranch privileges_info;
     ASSERT_OK(sentry_authz_provider_->fetcher_.GetSentryPrivileges(
-        requested_scope, table_ident, kTestUser, &privileges_info));
+        requested_scope, table_ident, kTestUser,
+        SentryCaching::ALL, &privileges_info));
     ASSERT_EQ(1, privileges_info.privileges().size());
     const auto& authorizable_privileges = *privileges_info.privileges().cbegin();
     ASSERT_EQ(GetParam(), authorizable_privileges.scope)
@@ -858,11 +860,11 @@ TEST_F(SentryAuthzProviderTest, CacheBehaviorScopeHierarchyAdjacentBranches) {
   ASSERT_EQ(0, GetTasksFailedNonFatal());
 }
 
-// Ensure requests for authorizables of the DATABASE scope hit cache once
+// Ensure requests to authorize CreateTables and AlterTables hit cache once
 // the information was fetched from Sentry for an authorizable of the TABLE
 // scope in the same hierarchy branch. A bit of context: Sentry sends all
 // available information for the branch up the authz scope hierarchy.
-TEST_F(SentryAuthzProviderTest, CacheBehaviorForDatabaseScope) {
+TEST_F(SentryAuthzProviderTest, CacheBehaviorForCreateAndAlter) {
   ASSERT_OK(CreateRoleAndAddToGroups());
   ASSERT_OK(AlterRoleGrantPrivilege(GetDatabasePrivilege("db0", "ALTER")));
   ASSERT_OK(AlterRoleGrantPrivilege(GetDatabasePrivilege("db1", "CREATE")));
@@ -1001,7 +1003,7 @@ TEST_F(SentryAuthzProviderTest, CacheBehaviorHybridLookups) {
 
 // Verify that information on TABLE-scope privileges are fetched from Sentry,
 // but not cached when SentryPrivilegeFetcher receives a ListPrivilegesByUser
-// response for a DATABASE-scope authorizable.
+// response for a DATABASE-scope authorizable for CreateTables or AlterTables.
 TEST_F(SentryAuthzProviderTest, CacheBehaviorNotCachingTableInfo) {
   ASSERT_OK(CreateRoleAndAddToGroups());
   ASSERT_OK(AlterRoleGrantPrivilege(GetDatabasePrivilege("db", "CREATE")));
diff --git a/src/kudu/master/sentry_authz_provider.cc b/src/kudu/master/sentry_authz_provider.cc
index bdc2fdf..feba6a2 100644
--- a/src/kudu/master/sentry_authz_provider.cc
+++ b/src/kudu/master/sentry_authz_provider.cc
@@ -52,7 +52,7 @@ namespace {
 // with GRANT ALL option, if any required ('requires_all_with_grant').
 bool IsActionAllowed(SentryAction::Action required_action,
                      SentryAuthorizableScope::Scope required_scope,
-                     bool requires_all_with_grant,
+                     SentryGrantRequired requires_all_with_grant,
                      const SentryPrivilegesBranch& privileges_branch) {
   // In general, a privilege implies another when:
   // 1. the authorizable from the former implies the authorizable from the latter
@@ -81,7 +81,7 @@ bool IsActionAllowed(SentryAction::Action required_action,
   for (const auto& privilege : privileges) {
     // A grant option cannot imply the other if the latter is set but the
     // former is not.
-    if (requires_all_with_grant && !privilege.all_with_grant) {
+    if (requires_all_with_grant == REQUIRED && !privilege.all_with_grant) {
       continue;
     }
     // Both privilege scope and action need to imply the other.
@@ -132,16 +132,20 @@ Status SentryAuthzProvider::AuthorizeCreateTable(const string& table_name,
   //
   // Otherwise, table creation requires 'CREATE ON DATABASE' privilege.
   SentryAction::Action action;
-  bool grant_option;
+  SentryGrantRequired grant_option;
   if (user == owner) {
     action = SentryAction::Action::CREATE;
-    grant_option = false;
+    grant_option = NOT_REQUIRED;
   } else {
     action = SentryAction::Action::ALL;
-    grant_option = true;
+    grant_option = REQUIRED;
   }
+  // Note: in our request to Sentry, we shouldn't cache table- or column-level
+  // privileges for the table, since Sentry may automatically grant privileges
+  // upon creation of new tables that caching might miss.
   return Authorize(SentryAuthorizableScope::Scope::DATABASE, action,
-                   table_name, user, grant_option);
+                   table_name, user, grant_option,
+                   SentryCaching::SERVER_AND_DB_ONLY);
 }
 
 Status SentryAuthzProvider::AuthorizeDropTable(const string& table_name,
@@ -170,9 +174,13 @@ Status SentryAuthzProvider::AuthorizeAlterTable(const string& old_table,
   RETURN_NOT_OK(Authorize(SentryAuthorizableScope::Scope::TABLE,
                           SentryAction::Action::ALL,
                           old_table, user));
+  // Note: in our request to Sentry, we shouldn't cache table- or column-level
+  // privileges for the table, since Sentry may automatically alter privileges
+  // upon altering tables that caching might miss.
   return Authorize(SentryAuthorizableScope::Scope::DATABASE,
-                   SentryAction::Action::CREATE,
-                   new_table, user);
+                   SentryAction::Action::CREATE, new_table, user,
+                   SentryGrantRequired::NOT_REQUIRED,
+                   SentryCaching::SERVER_AND_DB_ONLY);
 }
 
 Status SentryAuthzProvider::AuthorizeGetTableMetadata(const string& table_name,
@@ -220,7 +228,8 @@ Status SentryAuthzProvider::FillTablePrivilegePB(const string& table_name,
   // be different upon subsequent calls to this function.
   SentryPrivilegesBranch privileges_branch;
   RETURN_NOT_OK(fetcher_.GetSentryPrivileges(
-      SentryAuthorizableScope::TABLE, table_name, user, &privileges_branch));
+      SentryAuthorizableScope::TABLE, table_name, user,
+      SentryCaching::ALL, &privileges_branch));
   unordered_set<string> scannable_col_names;
   static const SentryAuthorizableScope kTableScope(SentryAuthorizableScope::TABLE);
   for (const auto& privilege : privileges_branch.privileges()) {
@@ -273,13 +282,15 @@ Status SentryAuthzProvider::Authorize(SentryAuthorizableScope::Scope scope,
                                       SentryAction::Action action,
                                       const string& table_ident,
                                       const string& user,
-                                      bool require_grant_option) {
+                                      SentryGrantRequired require_grant_option,
+                                      SentryCaching caching) {
   if (AuthzProvider::IsTrustedUser(user)) {
     return Status::OK();
   }
 
   SentryPrivilegesBranch privileges;
-  RETURN_NOT_OK(fetcher_.GetSentryPrivileges(scope, table_ident, user, &privileges));
+  RETURN_NOT_OK(fetcher_.GetSentryPrivileges(scope, table_ident, user,
+      caching, &privileges));
   if (IsActionAllowed(action, scope, require_grant_option, privileges)) {
     return Status::OK();
   }
diff --git a/src/kudu/master/sentry_authz_provider.h b/src/kudu/master/sentry_authz_provider.h
index c23e74f..7f1496f 100644
--- a/src/kudu/master/sentry_authz_provider.h
+++ b/src/kudu/master/sentry_authz_provider.h
@@ -41,6 +41,13 @@ class TablePrivilegePB;
 
 namespace master {
 
+// Enum indicating whether a grant option is required to perform a specific
+// action.
+enum SentryGrantRequired {
+  NOT_REQUIRED,
+  REQUIRED,
+};
+
 // An implementation of AuthzProvider that connects to the Sentry service
 // for authorization metadata and allows or denies the actions performed by
 // users based on the metadata.
@@ -109,11 +116,16 @@ class SentryAuthzProvider : public AuthzProvider {
   // If the operation is not authorized, returns Status::NotAuthorized().
   // Note that the authorization process is case insensitive for the
   // authorizables.
+  //
+  // If 'caching' is SERVER_AND_DB_ONLY and the underlying
+  // SentryPrivilegesFetcher is configured to cache privileges, it will not
+  // cache privileges equal to or below the 'TABLE' scope.
   Status Authorize(sentry::SentryAuthorizableScope::Scope scope,
                    sentry::SentryAction::Action action,
                    const std::string& table_ident,
                    const std::string& user,
-                   bool require_grant_option = false);
+                   SentryGrantRequired require_grant_option = NOT_REQUIRED,
+                   SentryCaching caching = ALL);
 
   // An instance of utility class that provides interface to search for
   // required privileges through the information received from Sentry.
diff --git a/src/kudu/master/sentry_privileges_fetcher.cc b/src/kudu/master/sentry_privileges_fetcher.cc
index ff04a1f..f293610 100644
--- a/src/kudu/master/sentry_privileges_fetcher.cc
+++ b/src/kudu/master/sentry_privileges_fetcher.cc
@@ -562,7 +562,7 @@ void SentryPrivilegesFetcher::Stop() {
 Status SentryPrivilegesFetcher::ResetCache() {
   const auto cache_capacity_bytes =
       FLAGS_sentry_privileges_cache_capacity_mb * 1024 * 1024;
-  shared_ptr<AuthzInfoCache> new_cache;
+  shared_ptr<PrivilegeCache> new_cache;
   if (cache_capacity_bytes != 0) {
     const auto cache_entry_ttl = MonoDelta::FromSeconds(
         FLAGS_authz_token_validity_seconds *
@@ -574,7 +574,7 @@ Status SentryPrivilegesFetcher::ResetCache() {
           FLAGS_sentry_privileges_cache_scrubbing_period_sec));
     }
 
-    new_cache = make_shared<AuthzInfoCache>(
+    new_cache = make_shared<PrivilegeCache>(
         cache_capacity_bytes, cache_entry_ttl, cache_scrubbing_period,
         FLAGS_sentry_privileges_cache_max_scrubbed_entries_per_pass,
         "sentry-privileges-ttl-cache");
@@ -596,6 +596,7 @@ Status SentryPrivilegesFetcher::GetSentryPrivileges(
     SentryAuthorizableScope::Scope requested_scope,
     const string& table_ident,
     const string& user,
+    SentryCaching caching,
     SentryPrivilegesBranch* privileges) {
   Slice db_slice;
   Slice table_slice;
@@ -605,6 +606,8 @@ Status SentryPrivilegesFetcher::GetSentryPrivileges(
   const string table = table_slice.ToString();
   const string db = db_slice.ToString();
 
+  // 1. Put together the requested authorizable.
+
   TSentryAuthorizable authorizable;
   RETURN_NOT_OK(GetAuthorizable(db, table, requested_scope, &authorizable));
 
@@ -623,37 +626,39 @@ Status SentryPrivilegesFetcher::GetSentryPrivileges(
   // even in tests.
   DCHECK_NE(SentryAuthorizableScope::COLUMN, requested_scope);
 
-  const AuthzInfoKey aggregate_key(user, authorizable);
+  const AuthzInfoKey requested_info(user, authorizable);
   // Do not query Sentry for authz scopes narrower than 'TABLE'.
-  const auto& key = aggregate_key.GetKey(SentryAuthorizableScope::TABLE);
-  const auto& key_seq = aggregate_key.key_sequence();
+  const auto& requested_key = requested_info.GetKey(SentryAuthorizableScope::TABLE);
+  const auto& requested_key_seq = requested_info.key_sequence();
+
+  // 2. Check the cache to see if it contains the requested privileges.
 
   // Copy the shared pointer to the cache. That's necessary because:
   //   * the cache_ member may be reset by concurrent ResetCache()
   //   * TTLCache is based on Cache that doesn't allow for outstanding handles
   //     if the cache itself destructed (in this case, goes out of scope).
-  shared_ptr<AuthzInfoCache> cache;
+  shared_ptr<PrivilegeCache> cache;
   {
     shared_lock<rw_spinlock> l(cache_lock_);
     cache = cache_;
   }
-  vector<typename AuthzInfoCache::EntryHandle> handles;
+  vector<typename PrivilegeCache::EntryHandle> handles;
   handles.reserve(AuthzInfoKey::kKeySequenceMaxSize);
 
   if (PREDICT_TRUE(cache)) {
-    for (const auto& e : key_seq) {
+    for (const auto& e : requested_key_seq) {
       auto handle = cache->Get(e);
-      VLOG(3) << Substitute("'$0': '$1' key lookup", key, e);
+      VLOG(3) << Substitute("'$0': '$1' key lookup", requested_key, e);
       if (!handle) {
         continue;
       }
-      VLOG(2) << Substitute("'$0': '$1' key found", key, e);
+      VLOG(2) << Substitute("'$0': '$1' key found", requested_key, e);
       handles.emplace_back(std::move(handle));
     }
   }
   // If the cache contains all the necessary information, repackage the
   // cached information and return as the result.
-  if (handles.size() == key_seq.size()) {
+  if (handles.size() == requested_key_seq.size()) {
     SentryPrivilegesBranch result;
     for (const auto& e : handles) {
       DCHECK(e);
@@ -663,6 +668,15 @@ Status SentryPrivilegesFetcher::GetSentryPrivileges(
     return Status::OK();
   }
 
+  // 3. The required privileges do not exist in the cache. Fetch them from
+  // Sentry.
+
+  // Narrow the scope of the authorizable to limit the number of privileges
+  // sent back from Sentry to be relevant to the provided table.
+  NarrowAuthzScopeForFetch(db, table, &authorizable);
+  const AuthzInfoKey full_authz_info(user, authorizable);
+  const string& full_key = full_authz_info.GetKey(SentryAuthorizableScope::TABLE);
+
   Synchronizer sync;
   bool is_first_request = false;
   // The result (i.e. the retrieved informaton on privileges) might be used
@@ -672,7 +686,7 @@ Status SentryPrivilegesFetcher::GetSentryPrivileges(
   {
     std::lock_guard<simple_spinlock> l(pending_requests_lock_);
     auto& pending_request = LookupOrEmplace(&pending_requests_,
-                                            key, SentryRequestsInfo());
+                                            full_key, SentryRequestsInfo());
     // Is the queue of pending requests for the same key empty?
     // If yes, that's the first request being sent out.
     is_first_request = pending_request.callbacks.empty();
@@ -690,11 +704,12 @@ Status SentryPrivilegesFetcher::GetSentryPrivileges(
     return Status::OK();
   }
 
-  NarrowAuthzScopeForFetch(db, table, &authorizable);
   TRACE("Fetching privileges from Sentry");
   const auto s = FetchPrivilegesFromSentry(FLAGS_kudu_service_name,
                                            user, authorizable,
                                            fetched_privileges.get());
+
+  // 4. Cache the privileges from Sentry.
   if (s.ok() && PREDICT_TRUE(cache)) {
     // Put the result into the cache. Negative results (i.e. errors) are not
     // cached. Split the information on privileges into at most two cache
@@ -709,43 +724,36 @@ Status SentryPrivilegesFetcher::GetSentryPrivileges(
     SentryPrivilegesBranch priv_table_column;
     fetched_privileges->Split(&priv_srv_db, &priv_table_column);
     if (requested_scope != SentryAuthorizableScope::SERVER) {
-      unique_ptr<SentryPrivilegesBranch> result_ptr(
-          new SentryPrivilegesBranch(std::move(priv_srv_db)));
-      const auto& key = aggregate_key.GetKey(
-          SentryAuthorizableScope::DATABASE);
-      const auto result_footprint = result_ptr->memory_footprint() +
-          key.capacity();
-      cache->Put(key, std::move(result_ptr), result_footprint);
-      VLOG(2) << Substitute(
-          "added entry of size $0 bytes for key '$1' (server-database scope)",
-          result_footprint, key);
-    }
-    // Don't add table-level records from the response into the cache
-    // if the request to Sentry was of database or higher scope. Due to the
-    // tree-like structure of Sentry's responses, those responses might contain
-    // information on non-Kudu tables which are not relevant in the context of
-    // Kudu's AuthzProvider. Upon detecting a cache miss for a table-scope key,
-    // the fetcher requests information on corresponding table-scope privileges
-    // from Sentry explicitly.
-    if ((requested_scope != SentryAuthorizableScope::SERVER) &&
-        (requested_scope != SentryAuthorizableScope::DATABASE)) {
-      unique_ptr<SentryPrivilegesBranch> result_ptr(
-          new SentryPrivilegesBranch(std::move(priv_table_column)));
-      const auto& key = aggregate_key.GetKey(
-          SentryAuthorizableScope::TABLE);
-      const auto result_footprint = result_ptr->memory_footprint() +
-          key.capacity();
-      cache->Put(key, std::move(result_ptr), result_footprint);
-      VLOG(2) << Substitute(
-          "added entry of size $0 bytes for key '$1' (table-column scope)",
-          result_footprint, key);
+      {
+        unique_ptr<SentryPrivilegesBranch> result_ptr(
+            new SentryPrivilegesBranch(std::move(priv_srv_db)));
+        const auto& db_key = full_authz_info.GetKey(SentryAuthorizableScope::DATABASE);
+        const auto result_footprint =
+            result_ptr->memory_footprint() + db_key.capacity();
+        cache->Put(db_key, std::move(result_ptr), result_footprint);
+        VLOG(2) << Substitute(
+            "added entry of size $0 bytes for key '$1' (server-database scope)",
+            result_footprint, db_key);
+      }
+      if (caching == ALL) {
+        unique_ptr<SentryPrivilegesBranch> result_ptr(
+            new SentryPrivilegesBranch(std::move(priv_table_column)));
+        const auto& table_key = full_authz_info.GetKey(SentryAuthorizableScope::TABLE);
+        const auto result_footprint =
+            result_ptr->memory_footprint() + table_key.capacity();
+        cache->Put(table_key, std::move(result_ptr), result_footprint);
+        VLOG(2) << Substitute(
+            "added entry of size $0 bytes for key '$1' (table-column scope)",
+            result_footprint, table_key);
+      }
     }
   }
 
+  // 5. Run any pending callbacks and return.
   SentryRequestsInfo info;
   {
     std::lock_guard<simple_spinlock> l(pending_requests_lock_);
-    info = EraseKeyReturnValuePtr(&pending_requests_, key);
+    info = EraseKeyReturnValuePtr(&pending_requests_, full_key);
   }
   CHECK_LE(1, info.callbacks.size());
   for (auto& cb : info.callbacks) {
diff --git a/src/kudu/master/sentry_privileges_fetcher.h b/src/kudu/master/sentry_privileges_fetcher.h
index a1b704c..a43732c 100644
--- a/src/kudu/master/sentry_privileges_fetcher.h
+++ b/src/kudu/master/sentry_privileges_fetcher.h
@@ -50,6 +50,11 @@ class TSentryPrivilege;
 namespace kudu {
 namespace master {
 
+enum SentryCaching {
+  ALL,
+  SERVER_AND_DB_ONLY,
+};
+
 // Utility struct to facilitate evaluating the privileges of a given
 // authorizable. This is preferred to using Sentry's Thrift responses directly,
 // since useful information has already been parsed to generate this struct
@@ -167,10 +172,15 @@ class SentryPrivilegesFetcher {
   // by the given table and scope. The result privileges might be served
   // from the cache, if caching is enabled and corresponding entry exists
   // in the cache.
+  //
+  // If 'caching' is SERVER_AND_DB_ONLY and the SentryPrivilegesFetcher is
+  // configured to cache privileges, it will not cache privileges equal to or
+  // below the 'TABLE' scope.
   Status GetSentryPrivileges(
       sentry::SentryAuthorizableScope::Scope requested_scope,
       const std::string& table_ident,
       const std::string& user,
+      SentryCaching caching,
       SentryPrivilegesBranch* privileges);
 
  private:
@@ -222,8 +232,8 @@ class SentryPrivilegesFetcher {
   // The TTL cache to store information on privileges received from Sentry.
   // The instance is wrapped into std::shared_ptr to handle operations with
   // cache items along with concurrent requests to reset the instance.
-  typedef TTLCache<std::string, SentryPrivilegesBranch> AuthzInfoCache;
-  std::shared_ptr<AuthzInfoCache> cache_;
+  typedef TTLCache<std::string, SentryPrivilegesBranch> PrivilegeCache;
+  std::shared_ptr<PrivilegeCache> cache_;
 
   // Synchronization primitive to guard access to the cache in the presence
   // of operations with cache items and concurrent requests to reset the cache.