You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2019/04/25 16:34:26 UTC

[kudu] branch master updated: [authz] new SentryAuthzProvider's caching strategy

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

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


The following commit(s) were added to refs/heads/master by this push:
     new a10d025  [authz] new SentryAuthzProvider's caching strategy
a10d025 is described below

commit a10d025dc13012c1a387de0607f1bd554b2038c0
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Tue Apr 16 10:14:08 2019 -0700

    [authz] new SentryAuthzProvider's caching strategy
    
    This patch updates the way how the privilege cache in
    SentryAuthzProvider is populated.  Prior to this patch, only one entry
    per sanitized Sentry's response was created.  With this patch,
    a response may be split into two entries: one contains server- and
    database-scope privileges, and another contains table- and column-scope
    privileges.  Of course, it also changes the lookup process: now it's
    necessary to search for two entries in the cache if looking up for
    an entry with privileges for an authorizable of the table scope.
    
    The new caching strategy leverages the fact that Sentry includes
    information on privileges granted on authorizables of higher scopes
    in the hierarchy, if any.  The new strategy is beneficial in cases
    when a user has privileges granted on database.  In that case, once
    there was a request to authorize an action on a table or a column
    of that table, next request to authorize an action on the database
    itself will hit the cache, avoiding an extra RPC sent to Sentry.
    The other scenario that benefits from the new caching scheme is a
    a sequence of AuthorizeDropTable() followed by AuthorizeCreateTable().
    
    See below for more details.
    
    The API that Kudu uses to fetch privileges from Sentry returns
    privileges corresponding to all ancestors and descendants of the
    requested authorizable in Sentry's scope hierarchy tree.  For instance,
    if requesting database-level privileges on "s.d", the API returns
    privileges on "s", "s.d", privileges on all tables within "s.d",
    and privileges on all columns therein if any of those privileges are
    granted.  In case of many tables in a single database we would end up
    caching information for all of them, while among those tables there
    might be many of non-Kudu tables.
    
    Additionally, the existing cache-lookup policy doesn't allow for
    leveraging of the information that exist in the cache where it could.
    For example, with prior strategy the following might be stored in
    the cache:
    
      { /s/d/t => [ "METADATA" on /s,
                    "CREATE"   on /s/d,
                    "SELECT"   on /s/d/t,
                    "UPDATE"   on /s/d/t/c, ] }
    
    At the same time, a cache lookup for privileges on "/s/d" would yield an
    expensive database-scoped request to Sentry (expensive in terms of
    amount of data returned by Sentry in case of database with many tables).
    If we were smarter about how we structured our privileges in cache,
    we could return the cached database-scoped privilege.
    
    This patch addresses these problems by splitting the cached privileges
    into at most two entries: those for database-scope and above (keyed
    by database), and those for table-scope and below (keyed by table).
    For cache lookups for a given authorizable, this allows us to use
    existing privileges for higher scopes from other authorizables where
    possible, avoiding expensive calls to Sentry for database-scoped
    privileges.  For the example above, with the new caching strategy,
    the cache will store the following entries:
    
      { /s/d   => [ "METADATA" on /s,
                    "CREATE"   on /s/d, ] }
    
      { /s/d/t => [ "SELECT"   on /s/d/t,
                    "UPDATE"   on /s/d/t/c, ] }
    
    With the above in cache, a lookup on /s/d would thus be able to leverage
    the existing entry.  The trade-off is that:
      * every Sentry's response on table-scope privileges is stored as
        two entries in the cache: one for database-scope and another
        one for table-scope
      * fetching of table-scope information require two cache lookups
        (for example above one for /s/d and another for /s/d/t key)
    
    Why not to split a response further, using 'one entry per each
    authz-scope' approach?  Based on current privilege model, Kudu requests
    privileges at the database scope (e.g. for CreateTable) or table scope
    (e.g. OpenTable) only, so this separation seems natural while limiting
    the number of cache entries to create/lookup.
    
    To address the concern about caching irrelevant information on non-Kudu
    tables sent by Sentry with a _database-scope_ response, with the new
    caching strategy the information on table- and column-scope
    authorizables is chopped off from database-scope responses before
    storing corresponding database-level entry in the cache.  That's also
    necessary from the point of keeping information in the cache consistent
    while avoiding splitting a single Sentry response into unlimited number
    of entries in the cache.  Otherwise, it will be necessary to put
    information on each table-scope sub-tree into a separate cache entry
    to leverage the presence of the whole database sub-tree in a Sentry's
    response.
    
    Change-Id: Id96181345e357a104e28314d8d8d88633dcf9608
    Reviewed-on: http://gerrit.cloudera.org:8080/13069
    Tested-by: Alexey Serbin <as...@cloudera.com>
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Reviewed-by: Hao Hao <ha...@cloudera.com>
---
 src/kudu/master/sentry_authz_provider-test.cc | 195 +++++++++++++++++-
 src/kudu/master/sentry_privileges_fetcher.cc  | 271 +++++++++++++++++++++-----
 src/kudu/master/sentry_privileges_fetcher.h   |  16 +-
 3 files changed, 423 insertions(+), 59 deletions(-)

diff --git a/src/kudu/master/sentry_authz_provider-test.cc b/src/kudu/master/sentry_authz_provider-test.cc
index 8d015cc..b70ea24 100644
--- a/src/kudu/master/sentry_authz_provider-test.cc
+++ b/src/kudu/master/sentry_authz_provider-test.cc
@@ -501,7 +501,7 @@ class SentryAuthzProviderFilterPrivilegesScopeTest :
     public SentryAuthzProviderFilterPrivilegesTest,
     public ::testing::WithParamInterface<SentryAuthorizableScope::Scope> {};
 
-// Attempst to grant privileges for various actions on a single scope of an
+// Attempt to grant privileges for various actions on a single scope of an
 // authorizable, injecting various invalid privileges, and checking that Kudu
 // ignores them.
 TEST_P(SentryAuthzProviderFilterPrivilegesScopeTest, TestFilterInvalidResponses) {
@@ -653,12 +653,13 @@ TEST_F(SentryAuthzProviderTest, TestAuthorizeAlterTable) {
 }
 
 TEST_F(SentryAuthzProviderTest, TestAuthorizeGetTableMetadata) {
-  // Don't authorize delete table on a user without required privileges.
+  // Don't authorize getting metadata on a table for a user without required
+  // privileges.
   ASSERT_OK(CreateRoleAndAddToGroups());
   Status s = sentry_authz_provider_->AuthorizeGetTableMetadata("db.table", kTestUser);
   ASSERT_TRUE(s.IsNotAuthorized()) << s.ToString();
 
-  // Authorize delete table on a user with proper privileges.
+  // Authorize getting metadata on a table for a user with proper privileges.
   TSentryPrivilege privilege = GetDatabasePrivilege("db", "SELECT");
   ASSERT_OK(AlterRoleGrantPrivilege(privilege));
   ASSERT_OK(sentry_authz_provider_->AuthorizeGetTableMetadata("db.table", kTestUser));
@@ -692,6 +693,188 @@ TEST_F(SentryAuthzProviderTest, TestPrivilegeCaseSensitivity) {
   ASSERT_OK(sentry_authz_provider_->AuthorizeCreateTable("DB.table", kTestUser, kTestUser));
 }
 
+// Verify the behavior of the SentryAuthzProvider's cache upon fetching
+// privilege information on authorizables of the TABLE scope in
+// 'adjacent branches' of the authz hierarchy.
+TEST_F(SentryAuthzProviderTest, CacheBehaviorScopeHierarchyAdjacentBranches) {
+  ASSERT_OK(CreateRoleAndAddToGroups());
+  ASSERT_OK(AlterRoleGrantPrivilege(GetDatabasePrivilege("db", "METADATA")));
+  ASSERT_OK(AlterRoleGrantPrivilege(GetTablePrivilege("db", "t0", "ALTER")));
+  ASSERT_OK(AlterRoleGrantPrivilege(GetTablePrivilege("db", "t1", "ALTER")));
+
+  // ALTER TABLE, if not renaming the table itself, requires ALTER privilege
+  // on the table, but nothing is required on the database that contains
+  // the table. METADATA requires corresponding privilege on the table, but
+  // nothing is required on the database.
+  ASSERT_EQ(0, GetTasksSuccessful());
+  ASSERT_OK(sentry_authz_provider_->AuthorizeAlterTable(
+      "db.t0", "db.t0", kTestUser));
+  ASSERT_EQ(1, GetTasksSuccessful());
+  ASSERT_OK(sentry_authz_provider_->AuthorizeAlterTable(
+      "db.t1", "db.t1", kTestUser));
+  ASSERT_EQ(2, GetTasksSuccessful());
+  ASSERT_OK(sentry_authz_provider_->AuthorizeGetTableMetadata(
+      "db.other_table", kTestUser));
+  ASSERT_EQ(3, GetTasksSuccessful());
+
+  // Repeat all the requests above: not a single new RPC to Sentry should be
+  // sent since all authz queries must hit the cache: that's about repeating
+  // the same requests.
+  ASSERT_OK(sentry_authz_provider_->AuthorizeAlterTable(
+      "db.t0", "db.t0", kTestUser));
+  ASSERT_OK(sentry_authz_provider_->AuthorizeAlterTable(
+      "db.t1", "db.t1", kTestUser));
+  ASSERT_OK(sentry_authz_provider_->AuthorizeGetTableMetadata(
+      "db.other_table", kTestUser));
+  ASSERT_EQ(3, GetTasksSuccessful());
+
+  // All the requests below should also hit the cache since the information on
+  // the privileges granted on each of the tables in the requests below
+  // is in the cache. In the Sentry's privileges model for Kudu, DROP TABLE
+  // requires privileges on the table itself, but nothing is required on the
+  // database the table belongs to.
+  Status s = sentry_authz_provider_->AuthorizeDropTable("db.t0", kTestUser);
+  ASSERT_TRUE(s.IsNotAuthorized()) << s.ToString();
+  s = sentry_authz_provider_->AuthorizeDropTable("db.t1", kTestUser);
+  ASSERT_TRUE(s.IsNotAuthorized()) << s.ToString();
+  s = sentry_authz_provider_->AuthorizeDropTable("db.other_table", kTestUser);
+  ASSERT_TRUE(s.IsNotAuthorized()) << s.ToString();
+  ASSERT_EQ(3, GetTasksSuccessful());
+
+  // A sanity check: verify no failed requests are registered.
+  ASSERT_EQ(0, GetTasksFailedFatal());
+  ASSERT_EQ(0, GetTasksFailedNonFatal());
+}
+
+// Ensure requests for authorizables of the DATABASE scope 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) {
+  ASSERT_OK(CreateRoleAndAddToGroups());
+  ASSERT_OK(AlterRoleGrantPrivilege(GetDatabasePrivilege("db0", "ALTER")));
+  ASSERT_OK(AlterRoleGrantPrivilege(GetDatabasePrivilege("db1", "CREATE")));
+  ASSERT_EQ(0, GetTasksSuccessful());
+
+  // ALTER TABLE, if not renaming the table itself, requires privileges on the
+  // table only.
+  ASSERT_OK(sentry_authz_provider_->AuthorizeAlterTable(
+      "db0.t0", "db0.t0", kTestUser));
+  ASSERT_EQ(1, GetTasksSuccessful());
+
+  // The CREATE privileges is not granted on the 'db0', so the request must
+  // not be authorized.
+  auto s = sentry_authz_provider_->AuthorizeCreateTable(
+      "db0.t1", kTestUser, kTestUser);
+  ASSERT_TRUE(s.IsNotAuthorized());
+  // CREATE TABLE requires privileges on the database only, and those should
+  // have been cached already due to the prior request.
+  ASSERT_EQ(1, GetTasksSuccessful());
+
+  // No new RPCs to Sentry should be issued: the information on privileges
+  // on 'db1' authorizable of the DATABASE scope should be fetched and cached
+  // while fetching the information privileges on 'db1.t0' authorizable of the
+  // TABLE scope.
+  for (int idx = 0; idx < 10; ++idx) {
+    const auto table_name = Substitute("db1.t$0", idx);
+    ASSERT_OK(sentry_authz_provider_->AuthorizeCreateTable(
+        table_name, kTestUser, kTestUser));
+  }
+  // Only a single new RPC should be issued to Sentry: to get information
+  // for "db1" authorizable of the DATABASE scope while authorizing the creation
+  // of table "db1.t0". All other requests must hit the cache.
+  ASSERT_EQ(2, GetTasksSuccessful());
+
+  // Same story for requests for 'db1.t0', ..., 'db1.t19'.
+  for (int idx = 0; idx < 20; ++idx) {
+    const auto table_name = Substitute("db1.t$0", idx);
+    ASSERT_OK(sentry_authz_provider_->AuthorizeCreateTable(
+        table_name, kTestUser, kTestUser));
+  }
+  ASSERT_EQ(2, GetTasksSuccessful());
+
+  // A sanity check: verify no failed requests are registered.
+  ASSERT_EQ(0, GetTasksFailedFatal());
+  ASSERT_EQ(0, GetTasksFailedNonFatal());
+}
+
+// A scenario where a TABLE-scope privilege on a table is granted to a user,
+// but there isn't any DATABASE-scope privilege granted on the database
+// the table belongs to. After authorizing an operation on the table, there
+// should not be another RPC to Sentry issued while authorizing an operation
+// on the database itself.
+TEST_F(SentryAuthzProviderTest, CacheBehaviorHybridLookups) {
+  ASSERT_OK(CreateRoleAndAddToGroups());
+  ASSERT_OK(AlterRoleGrantPrivilege(GetTablePrivilege("db", "t", "ALL")));
+
+  ASSERT_EQ(0, GetTasksSuccessful());
+  // In the Sentry's authz model for Kudu, DROP TABLE requires only privileges
+  // on the table itself.
+  ASSERT_OK(sentry_authz_provider_->AuthorizeDropTable("db.t", kTestUser));
+  ASSERT_EQ(1, GetTasksSuccessful());
+
+  // CREATE TABLE requires privileges only on the database itself. No privileges
+  // are granted on the database, so the request must not be authorized.
+  auto s = sentry_authz_provider_->AuthorizeCreateTable(
+      "db.t", kTestUser, kTestUser);
+  ASSERT_TRUE(s.IsNotAuthorized()) << s.ToString();
+  // No extra RPC should be sent to Sentry: the information on the privileges
+  // granted on relevant authorizables of the DATABASE scope in corresponding
+  // branch should have been fetched and cached.
+  ASSERT_EQ(1, GetTasksSuccessful());
+
+  // ALTER TABLE, if renaming the table, requires privileges both on the
+  // database and the table. Even if ALL is granted on the table itself, there
+  // isn't any privilege granted on the database, so the request to rename
+  // the table must not be authorized.
+  s = sentry_authz_provider_->AuthorizeAlterTable(
+      "db.t", "db.t_renamed", kTestUser);
+  ASSERT_TRUE(s.IsNotAuthorized()) << s.ToString();
+  // No extra RPCs are expected in this case.
+  ASSERT_EQ(1, GetTasksSuccessful());
+
+  // A sanity check: verify no failed requests are registered.
+  ASSERT_EQ(0, GetTasksFailedFatal());
+  ASSERT_EQ(0, GetTasksFailedNonFatal());
+}
+
+// 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.
+TEST_F(SentryAuthzProviderTest, CacheBehaviorNotCachingTableInfoOnDatabase) {
+  ASSERT_OK(CreateRoleAndAddToGroups());
+  ASSERT_OK(AlterRoleGrantPrivilege(GetDatabasePrivilege("db", "CREATE")));
+  ASSERT_OK(AlterRoleGrantPrivilege(GetTablePrivilege("db", "t0", "ALL")));
+  ASSERT_OK(AlterRoleGrantPrivilege(GetTablePrivilege("db", "t1", "ALTER")));
+
+  ASSERT_EQ(0, GetTasksSuccessful());
+  // In the Sentry's authz model for Kudu, CREATE TABLE requires only privileges
+  // only on the database.
+  ASSERT_OK(sentry_authz_provider_->AuthorizeCreateTable(
+      "db.table", kTestUser, kTestUser));
+  ASSERT_EQ(1, GetTasksSuccessful());
+
+  // ALTER TABLE, if not renaming the table, requires privileges only on the
+  // table itself.
+  ASSERT_OK(sentry_authz_provider_->AuthorizeAlterTable(
+      "db.t0", "db.t0", kTestUser));
+  // Here an RPC request should be sent to Sentry to fetch information on
+  // privileges granted at the table 'db.t0'. That information was fetched
+  // from Sentry upon prior call to AuthorizeCreateTable(), but it was not
+  // cached deliberately: that way the cache avoids storing information on
+  // non-Kudu tables, if any, under an authorizable of the DATABASE scope.
+  ASSERT_EQ(2, GetTasksSuccessful());
+
+  // The same as above stays valid for the 'db.t1' table.
+  ASSERT_OK(sentry_authz_provider_->AuthorizeAlterTable(
+      "db.t1", "db.t1", kTestUser));
+  ASSERT_EQ(3, GetTasksSuccessful());
+
+  // A sanity check: verify no failed requests are registered.
+  ASSERT_EQ(0, GetTasksFailedFatal());
+  ASSERT_EQ(0, GetTasksFailedNonFatal());
+}
+
 // Whether the authz information received from Sentry is cached or not.
 enum class AuthzCaching {
   Disabled,
@@ -740,7 +923,11 @@ TEST_P(SentryAuthzProviderReconnectionTest, ConnectionFailureOrTooBusy) {
   }
 
   s = sentry_authz_provider_->AuthorizeCreateTable("db.table", kTestUser, "diff-user");
-  EXPECT_TRUE(s.IsNetworkError()) << s.ToString();
+  if (CachingEnabled()) {
+    EXPECT_TRUE(s.IsNotAuthorized()) << s.ToString();
+  } else {
+    EXPECT_TRUE(s.IsNetworkError()) << s.ToString();
+  }
 
   // Start Sentry back up and ensure that the same operations succeed.
   ASSERT_OK(StartSentry());
diff --git a/src/kudu/master/sentry_privileges_fetcher.cc b/src/kudu/master/sentry_privileges_fetcher.cc
index 6fdaad0..bf1ad0a 100644
--- a/src/kudu/master/sentry_privileges_fetcher.cc
+++ b/src/kudu/master/sentry_privileges_fetcher.cc
@@ -17,8 +17,10 @@
 
 #include "kudu/master/sentry_privileges_fetcher.h"
 
+#include <algorithm>
 #include <cstddef>
 #include <cstdint>
+#include <iterator>
 #include <memory>
 #include <mutex>
 #include <ostream>
@@ -51,6 +53,7 @@
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/net_util.h"
 #include "kudu/util/slice.h"
+#include "kudu/util/test_util_prod.h"
 #include "kudu/util/ttl_cache_metrics.h"
 
 DEFINE_string(sentry_service_rpc_addresses, "",
@@ -210,38 +213,78 @@ Status GetAuthorizable(const string& table_ident,
   return Status::OK();
 }
 
-// A utility class to help with Sentry privilege scoping, generating
-// sequence of keys to lookup corresponding entries in the cache.
-// For example, for user 'U' and authorizable { server:S, db:D, table:T }
-// (in pseudo-form) the sequence of keys is { 'U/S', 'U/S/D', 'U/S/D/T' }.
+// A utility class to help with Sentry privilege scoping, generating sequence
+// of keys to lookup corresponding entries in the cache.
 class AuthzInfoKey {
  public:
+  // The maximum possible number of the elements in the key lookup sequence
+  // returned by the key_sequence() method (see below). Maximum number of keys
+  // to lookup in the cache is 2. See the comment for the GenerateKeySequence()
+  // method below for more details.
+  constexpr static size_t kKeySequenceMaxSize = 2;
+
   AuthzInfoKey(const string& user,
                const ::sentry::TSentryAuthorizable& authorizable);
 
-  const string& GetFlattenedKey() {
-    // The flattened key is the very last element of the key_sequence_.
-    return key_sequence_.back();
+  // Get the key to lookup the corresponding entry in the cache with the scope
+  // of the authorizable widened as specified by the 'scope' parameter.
+  // E.g., if the original scope of the autorizable specified in the constructor
+  // was COLUMN, with the 'scope' set to TABLE the returned key is 'U/S/D/T',
+  // while the key for the authorizable as is would be 'U/S/D/T/C'.
+  const string& GetKey(SentryAuthorizableScope::Scope scope) const;
+
+  // This method returns the sequence of keys to look up in the cache
+  // if retrieving privileges granted to the 'user' on the 'authorizable'
+  // specified in the constructor.
+  const vector<string>& key_sequence() const {
+    return key_sequence_;
   }
 
  private:
-  // Generate the key lookup sequence: a sequence of keys to use while
-  // looking up corresponding entry in the authz cache based on the
-  // hierarchy of scoping for Sentry authorizables.
-  static vector<string> GenerateKeySequence(
+  // Generate the raw key sequence: a sequence of keys for the authz scope
+  // hierarchy, starting from the very top (i.e. SERVER scope) and narrowing
+  // down to the scope of the 'authorizable' specified in the constructor.
+  //
+  // For example, for user 'U' and authorizable { server:S, db:D, table:T }
+  // the raw sequence of keys is { 'U/S', 'U/S/D', 'U/S/D/T' }.
+  static vector<string> GenerateRawKeySequence(
       const string& user, const ::sentry::TSentryAuthorizable& authorizable);
 
+  // Generate the cache key lookup sequence: a sequence of keys to use while
+  // looking up corresponding entry in the authz cache. The maximum
+  // length of the returned sequence is limited by kCacheKeySequenceMaxSize.
+  //
+  // For authorizables of the TABLE scope and narrower, it returns sequence
+  // { 'U/S/D', 'U/S/D/T' }. For authorizables of the DATABASE scope it returns
+  // { 'U/S/D' }. For authorizables of the SERVER scope it returns { 'U/S' }.
+  static vector<string> GenerateKeySequence(const vector<string>& raw_sequence);
+
+  // Convert the Sentry authz scope to an index in the list
+  // { SERVER, DATABASE, TABLE, COLUMN }.
+  static size_t ScopeToRawSequenceIdx(SentryAuthorizableScope::Scope scope);
+
+  const vector<string> raw_key_sequence_;
   const vector<string> key_sequence_;
 };
 
 AuthzInfoKey::AuthzInfoKey(const string& user,
                            const ::sentry::TSentryAuthorizable& authorizable)
-    : key_sequence_(GenerateKeySequence(user, authorizable)) {
-  CHECK(!key_sequence_.empty());
+    : raw_key_sequence_(GenerateRawKeySequence(user, authorizable)),
+      key_sequence_(GenerateKeySequence(raw_key_sequence_)) {
+  DCHECK(!raw_key_sequence_.empty());
+  DCHECK(!key_sequence_.empty());
+  DCHECK_GE(kKeySequenceMaxSize, key_sequence_.size());
 }
 
-// TODO(aserbin): consider other ways of encoding a key for an object
-vector<string> AuthzInfoKey::GenerateKeySequence(
+const string& AuthzInfoKey::GetKey(SentryAuthorizableScope::Scope scope) const {
+  const size_t level = ScopeToRawSequenceIdx(scope);
+  if (level < raw_key_sequence_.size()) {
+    return raw_key_sequence_[level];
+  }
+  return raw_key_sequence_.back();
+}
+
+vector<string> AuthzInfoKey::GenerateRawKeySequence(
     const string& user, const ::sentry::TSentryAuthorizable& authorizable) {
   DCHECK(!user.empty());
   DCHECK(!authorizable.server.empty());
@@ -253,24 +296,66 @@ vector<string> AuthzInfoKey::GenerateKeySequence(
 
   if (!authorizable.__isset.table || authorizable.table.empty()) {
     auto k0 = Substitute("/$0/$1", user, authorizable.server);
-    auto k1 = Substitute("/$0/$1", k0, authorizable.db);
+    auto k1 = Substitute("$0/$1", k0, authorizable.db);
     return { std::move(k0), std::move(k1), };
   }
 
   if (!authorizable.__isset.column || authorizable.column.empty()) {
     auto k0 = Substitute("/$0/$1", user, authorizable.server);
-    auto k1 = Substitute("/$0/$1", k0, authorizable.db);
-    auto k2 = Substitute("/$0/$1", k1, authorizable.table);
+    auto k1 = Substitute("$0/$1", k0, authorizable.db);
+    auto k2 = Substitute("$0/$1", k1, authorizable.table);
     return { std::move(k0), std::move(k1), std::move(k2), };
   }
 
   auto k0 = Substitute("/$0/$1", user, authorizable.server);
-  auto k1 = Substitute("/$0/$1", k0, authorizable.db);
-  auto k2 = Substitute("/$0/$1", k1, authorizable.table);
-  auto k3 = Substitute("/$0/$1", k2, authorizable.column);
+  auto k1 = Substitute("$0/$1", k0, authorizable.db);
+  auto k2 = Substitute("$0/$1", k1, authorizable.table);
+  auto k3 = Substitute("$0/$1", k2, authorizable.column);
   return { std::move(k0), std::move(k1), std::move(k2), std::move(k3), };
 }
 
+vector<string> AuthzInfoKey::GenerateKeySequence(
+    const vector<string>& raw_sequence) {
+  DCHECK(!raw_sequence.empty());
+  vector<string> sequence;
+  const auto idx_db = ScopeToRawSequenceIdx(SentryAuthorizableScope::DATABASE);
+  if (idx_db < raw_sequence.size()) {
+    sequence.emplace_back(raw_sequence[idx_db]);
+  }
+  const auto idx_table = ScopeToRawSequenceIdx(SentryAuthorizableScope::TABLE);
+  if (idx_table < raw_sequence.size()) {
+    sequence.emplace_back(raw_sequence[idx_table]);
+  }
+  if (sequence.empty()) {
+    sequence.emplace_back(raw_sequence.back());
+  }
+  DCHECK_GE(kKeySequenceMaxSize, sequence.size());
+  return sequence;
+}
+
+size_t AuthzInfoKey::ScopeToRawSequenceIdx(SentryAuthorizableScope::Scope scope) {
+  size_t idx = 0;
+  switch (scope) {
+    case SentryAuthorizableScope::Scope::SERVER:
+      idx = 0;
+      break;
+    case SentryAuthorizableScope::Scope::DATABASE:
+      idx = 1;
+      break;
+    case SentryAuthorizableScope::Scope::TABLE:
+      idx = 2;
+      break;
+    case SentryAuthorizableScope::Scope::COLUMN:
+      idx = 3;
+      break;
+    default:
+      LOG(DFATAL) << "unexpected scope: " << static_cast<int16_t>(scope);
+      break;
+  }
+
+  return idx;
+}
+
 // Returns a unique string key for the given authorizable, at the given scope.
 // The authorizable must be a well-formed at the given scope.
 string GetKey(const string& server,
@@ -324,6 +409,35 @@ size_t SentryPrivilegesBranch::memory_footprint() const {
   return res;
 }
 
+void SentryPrivilegesBranch::Merge(const SentryPrivilegesBranch& other) {
+  std::copy(other.privileges_.begin(), other.privileges_.end(),
+            std::back_inserter(privileges_));
+}
+
+void SentryPrivilegesBranch::Split(
+    SentryPrivilegesBranch* other_scope_db,
+    SentryPrivilegesBranch* other_scope_table) const {
+  SentryPrivilegesBranch scope_db;
+  SentryPrivilegesBranch scope_table;
+  for (const auto& e : privileges_) {
+    switch (e.scope) {
+      case SentryAuthorizableScope::SERVER:
+      case SentryAuthorizableScope::DATABASE:
+        scope_db.privileges_.emplace_back(e);
+        break;
+      case SentryAuthorizableScope::TABLE:
+      case SentryAuthorizableScope::COLUMN:
+        scope_table.privileges_.emplace_back(e);
+        break;
+      default:
+        LOG(DFATAL) << "not reachable";
+        break;
+    }
+  }
+  *other_scope_db = std::move(scope_db);
+  *other_scope_table = std::move(scope_table);
+}
+
 void SentryPrivilegesBranch::DoInit(
     const ::sentry::TSentryAuthorizable& authorizable,
     const TListSentryPrivilegesResponse& response) {
@@ -336,7 +450,7 @@ void SentryPrivilegesBranch::DoInit(
       if (VLOG_IS_ON(1)) {
         std::ostringstream os;
         privilege_resp.printTo(os);
-        VLOG(1) << Substitute("Ignoring privilege response: $0", os.str());
+        VLOG(1) << Substitute("ignoring privilege response: $0", os.str());
       }
       continue;
     }
@@ -407,39 +521,63 @@ void SentryPrivilegesFetcher::Stop() {
   sentry_client_.Stop();
 }
 
-// TODO(aserbin): change the signature to return a handle that keeps reference
-//                to either cache entry or SentryPrivilegesBranch allocated
-//                on heap, otherwise there is copying from a cache entry to
-//                the output parameter.
 Status SentryPrivilegesFetcher::GetSentryPrivileges(
     SentryAuthorizableScope::Scope requested_scope,
     const string& table_name,
     const string& user,
     SentryPrivilegesBranch* privileges) {
-  // TODO(aserbin): once only requests with scope TABLE are issued,
-  //                uncomment the CHECK_EQ() below.
-  // Don't query Sentry for authz scopes other than 'TABLE'.
-  //CHECK_EQ(SentryAuthorizableScope::TABLE, requested_scope);
-  SentryAuthorizableScope scope(requested_scope);
   TSentryAuthorizable authorizable;
-  RETURN_NOT_OK(GetAuthorizable(table_name, scope.scope(), &authorizable));
+  RETURN_NOT_OK(GetAuthorizable(table_name, requested_scope, &authorizable));
+
+  if (PREDICT_FALSE(requested_scope == SentryAuthorizableScope::SERVER &&
+                    !IsGTest())) {
+    // A request for an authorizable of the scope wider than DATABASE is served,
+    // but the response from Sentry is not cached. With current privilege
+    // scheme, SentryPrivilegesFetcher is not expected to request authorizables
+    // of the SERVER scope unless this method is called from test code.
+    LOG(DFATAL) << Substitute(
+        "requesting privileges of the SERVER scope from Sentry "
+        "on authorizable '$0' for user '$1'", table_name, user);
+  }
 
-  AuthzInfoKey aggregate_key(user, authorizable);
-  const auto& key = aggregate_key.GetFlattenedKey();
-  typename AuthzInfoCache::EntryHandle handle;
+  // Not expecting requests for authorizables of the scope narrower than TABLE,
+  // even in tests.
+  DCHECK_NE(SentryAuthorizableScope::COLUMN, requested_scope);
+
+  const AuthzInfoKey aggregate_key(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();
+  vector<typename AuthzInfoCache::EntryHandle> handles;
+  handles.reserve(AuthzInfoKey::kKeySequenceMaxSize);
   if (PREDICT_TRUE(cache_)) {
-    handle = cache_->Get(key);
+    for (const auto& e : key_seq) {
+      auto handle = cache_->Get(e);
+      VLOG(3) << Substitute("'$0': '$1' key lookup", key, e);
+      if (!handle) {
+        continue;
+      }
+      VLOG(2) << Substitute("'$0': '$1' key found", key, e);
+      handles.emplace_back(std::move(handle));
+    }
   }
-  if (handle) {
-    *privileges = handle.value();
+  // If the cache contains all the necessary information, repackage the
+  // cached information and return as the result.
+  if (handles.size() == key_seq.size()) {
+    SentryPrivilegesBranch result;
+    for (const auto& e : handles) {
+      DCHECK(e);
+      result.Merge(e.value());
+    }
+    *privileges = std::move(result);
     return Status::OK();
   }
 
   Synchronizer sync;
   bool is_first_request = false;
-  // The result (i.e. fetched privileges) might be used independently
-  // by multiple threads. The shared ownership approach simplifies passing
-  // the information around.
+  // The result (i.e. the retrieved informaton on privileges) might be used
+  // independently by multiple threads. The shared ownership approach simplifies
+  // passing the information around.
   shared_ptr<SentryPrivilegesBranch> fetched_privileges;
   {
     std::lock_guard<simple_spinlock> l(pending_requests_lock_);
@@ -465,13 +603,50 @@ Status SentryPrivilegesFetcher::GetSentryPrivileges(
                                            user, authorizable,
                                            fetched_privileges.get());
   if (s.ok() && PREDICT_TRUE(cache_)) {
-    // Put the result into the cache. Negative results/errors are not cached.
-    unique_ptr<SentryPrivilegesBranch> result_ptr(
-        new SentryPrivilegesBranch(*fetched_privileges));
-    const auto result_footprint = result_ptr->memory_footprint() + key.capacity();
-    cache_->Put(key, std::move(result_ptr), result_footprint);
-    VLOG(2) << Substitute("cached entry of size $0 bytes for key '$1'",
-                          result_footprint, key);
+    // Put the result into the cache. Negative results (i.e. errors) are not
+    // cached. Split the information on privileges into at most two cache
+    // entries, for authorizables of scope:
+    //   * SERVER, DATABASE
+    //   * TABLE, COLUMN
+    //
+    // From this perspective, privileges on a corresponding authorizable of the
+    // DATABASE scope might be cached as a by-product when the original request
+    // comes for an authorizable of the TABLE scope.
+    SentryPrivilegesBranch priv_srv_db;
+    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);
+    }
   }
 
   SentryRequestsInfo info;
diff --git a/src/kudu/master/sentry_privileges_fetcher.h b/src/kudu/master/sentry_privileges_fetcher.h
index bed8f1c..71a7d75 100644
--- a/src/kudu/master/sentry_privileges_fetcher.h
+++ b/src/kudu/master/sentry_privileges_fetcher.h
@@ -25,7 +25,6 @@
 #include <utility>
 #include <vector>
 
-#include <boost/optional/optional.hpp>
 #include <glog/logging.h>
 #include <gtest/gtest_prod.h>
 
@@ -125,12 +124,20 @@ class SentryPrivilegesBranch {
   // Get estimation on amount of memory used (in bytes) to store this instance.
   size_t memory_footprint() const;
 
+  // Add/merge privileges from other instance of SentryPrivilegesBranch.
+  void Merge(const SentryPrivilegesBranch& other);
+
+  // Output the privileges into branches corresponding to DB-and-higher and
+  // TABLE-and-lower authz scopes.
+  void Split(SentryPrivilegesBranch* other_scope_db,
+             SentryPrivilegesBranch* other_scope_table) const;
+
  private:
   // Utility function.
   void DoInit(const ::sentry::TSentryAuthorizable& authorizable,
               const ::sentry::TListSentryPrivilegesResponse& response);
 
-  // Set of privileges are granted.
+  // Set of granted privileges.
   std::vector<AuthorizablePrivileges> privileges_;
 };
 
@@ -212,11 +219,6 @@ class SentryPrivilegesFetcher {
   // Metric entity for registering metric gauges/counters.
   scoped_refptr<MetricEntity> metric_entity_;
 
-  // The authz scope hierarchy level that defines the narrowest authz scope
-  // for requests sent to Sentry. If not set, no broadening of authz privilege
-  // scope is made.
-  boost::optional<sentry::SentryAuthorizableScope> scope_depth_limit_;
-
   // Client instance to communicate with Sentry.
   thrift::HaClient<sentry::SentryClient> sentry_client_;