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:50 UTC

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

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,