You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by ad...@apache.org on 2020/03/06 05:12:37 UTC

[kudu] 01/03: KUDU-2972 Add Ranger authorization provider

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

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

commit 0d29977c688ffeffb80785fe8966a4ffc3ee6a43
Author: Attila Bukor <ab...@apache.org>
AuthorDate: Thu Mar 5 22:44:22 2020 +0100

    KUDU-2972 Add Ranger authorization provider
    
    Adds a new authz_provider implementation that uses Ranger and plugs it
    in to catalog_manager. As of this patch this is considered experimental
    and an experimental flag is needed to be set to enable it.
    
    Change-Id: I6e7672a5947d6406e0cad83a0c900bf5b2c03012
    Reviewed-on: http://gerrit.cloudera.org:8080/15207
    Tested-by: Kudu Jenkins
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Reviewed-by: Hao Hao <ha...@cloudera.com>
---
 src/kudu/master/CMakeLists.txt           |   3 +
 src/kudu/master/catalog_manager.cc       |  18 +++
 src/kudu/master/ranger_authz_provider.cc | 202 +++++++++++++++++++++++++++++++
 src/kudu/master/ranger_authz_provider.h  |  91 ++++++++++++++
 src/kudu/master/sentry_authz_provider.cc |   5 +-
 5 files changed, 317 insertions(+), 2 deletions(-)

diff --git a/src/kudu/master/CMakeLists.txt b/src/kudu/master/CMakeLists.txt
index d4e27be..033429c 100644
--- a/src/kudu/master/CMakeLists.txt
+++ b/src/kudu/master/CMakeLists.txt
@@ -45,6 +45,7 @@ set(MASTER_SRCS
   master_service.cc
   mini_master.cc
   placement_policy.cc
+  ranger_authz_provider.cc
   sentry_authz_provider.cc
   sentry_client_metrics.cc
   sentry_privileges_cache_metrics.cc
@@ -62,7 +63,9 @@ target_link_libraries(master
   kserver
   kudu_common
   kudu_hms
+  kudu_ranger
   kudu_sentry
+  kudu_subprocess
   kudu_thrift
   kudu_util
   master_proto
diff --git a/src/kudu/master/catalog_manager.cc b/src/kudu/master/catalog_manager.cc
index 6aac7d8..8a2d8a5 100644
--- a/src/kudu/master/catalog_manager.cc
+++ b/src/kudu/master/catalog_manager.cc
@@ -106,6 +106,7 @@
 #include "kudu/master/master.pb.h"
 #include "kudu/master/master_cert_authority.h"
 #include "kudu/master/placement_policy.h"
+#include "kudu/master/ranger_authz_provider.h"
 #include "kudu/master/sentry_authz_provider.h"
 #include "kudu/master/sys_catalog.h"
 #include "kudu/master/table_metrics.h"
@@ -133,6 +134,7 @@
 #include "kudu/util/debug/trace_event.h"
 #include "kudu/util/fault_injection.h"
 #include "kudu/util/flag_tags.h"
+#include "kudu/util/flag_validators.h"
 #include "kudu/util/logging.h"
 #include "kudu/util/metrics.h"
 #include "kudu/util/monotime.h"
@@ -312,6 +314,20 @@ DECLARE_int64(tsk_rotation_seconds);
 
 METRIC_DEFINE_entity(table);
 
+DECLARE_string(sentry_service_rpc_addresses);
+DECLARE_string(ranger_config_path);
+
+static bool ValidateRangerSentryFlags() {
+  if (!FLAGS_sentry_service_rpc_addresses.empty() &&
+      !FLAGS_ranger_config_path.empty()) {
+    LOG(ERROR) << "--sentry_service_rpc_addresses and --ranger_config_path "
+                  "cannot be set at the same time.";
+    return false;
+  }
+  return true;
+}
+GROUP_FLAG_VALIDATOR(ranger_sentry_flags, ValidateRangerSentryFlags);
+
 using base::subtle::NoBarrier_CompareAndSwap;
 using base::subtle::NoBarrier_Load;
 using boost::make_optional;
@@ -752,6 +768,8 @@ CatalogManager::CatalogManager(Master* master)
       leader_lock_(RWMutex::Priority::PREFER_WRITING) {
   if (SentryAuthzProvider::IsEnabled()) {
     authz_provider_.reset(new SentryAuthzProvider(master_->metric_entity()));
+  } else if (RangerAuthzProvider::IsEnabled()) {
+    authz_provider_.reset(new RangerAuthzProvider(master_->metric_entity()));
   } else {
     authz_provider_.reset(new DefaultAuthzProvider);
   }
diff --git a/src/kudu/master/ranger_authz_provider.cc b/src/kudu/master/ranger_authz_provider.cc
new file mode 100644
index 0000000..5d92f7a
--- /dev/null
+++ b/src/kudu/master/ranger_authz_provider.cc
@@ -0,0 +1,202 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/master/ranger_authz_provider.h"
+
+#include <algorithm>
+#include <ostream>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/common/common.pb.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/ranger/ranger.pb.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/util/status.h"
+
+DECLARE_string(ranger_config_path);
+
+namespace kudu {
+class MetricEntity;
+namespace master {
+
+using kudu::security::ColumnPrivilegePB;
+using kudu::security::TablePrivilegePB;
+using kudu::ranger::ActionPB;
+using kudu::ranger::ActionHash;
+using std::string;
+using std::unordered_set;
+
+RangerAuthzProvider::RangerAuthzProvider(const scoped_refptr<MetricEntity>& metric_entity) :
+  client_(metric_entity) {}
+
+Status RangerAuthzProvider::Start() {
+  RETURN_NOT_OK(client_.Start());
+
+  return Status::OK();
+}
+
+Status RangerAuthzProvider::AuthorizeCreateTable(const string& table_name,
+                                                 const string& user,
+                                                 const string& /*owner*/) {
+  if (IsTrustedUser(user)) {
+    return Status::OK();
+  }
+  return client_.AuthorizeAction(user, ActionPB::CREATE, table_name);
+}
+
+Status RangerAuthzProvider::AuthorizeDropTable(const string& table_name,
+                                               const string& user) {
+  if (IsTrustedUser(user)) {
+    return Status::OK();
+  }
+  return client_.AuthorizeAction(user, ActionPB::DROP, table_name);
+}
+
+Status RangerAuthzProvider::AuthorizeAlterTable(const string& old_table,
+                                                const string& new_table,
+                                                const string& user) {
+  if (IsTrustedUser(user)) {
+    return Status::OK();
+  }
+  // Table alteration requires ALTER ON TABLE if no rename is done. To prevent
+  // privilege escalation we require ALL on the old table and CREATE on the new
+  // table (database).
+  if (old_table == new_table) {
+    return client_.AuthorizeAction(user, ActionPB::ALTER, old_table);
+  }
+
+  RETURN_NOT_OK(client_.AuthorizeAction(user, ActionPB::ALL, old_table));
+  return client_.AuthorizeAction(user, ActionPB::CREATE, new_table);
+}
+
+Status RangerAuthzProvider::AuthorizeGetTableMetadata(const string& table_name,
+                                                      const string& user) {
+  if (IsTrustedUser(user)) {
+    return Status::OK();
+  }
+  return client_.AuthorizeAction(user, ActionPB::METADATA, table_name);
+}
+
+Status RangerAuthzProvider::AuthorizeListTables(const string& user,
+                                                unordered_set<string>* table_names,
+                                                bool* checked_table_names) {
+  if (IsTrustedUser(user)) {
+    *checked_table_names = false;
+    return Status::OK();
+  }
+
+  *checked_table_names = true;
+  return client_.AuthorizeActionMultipleTables(user, ActionPB::METADATA, table_names);
+}
+
+Status RangerAuthzProvider::AuthorizeGetTableStatistics(const string& table_name,
+                                                        const string& user) {
+  if (IsTrustedUser(user)) {
+    return Status::OK();
+  }
+  // Statistics contain data (e.g. number of rows) that requires the 'SELECT ON TABLE'
+  // privilege.
+  return client_.AuthorizeAction(user, ActionPB::SELECT, table_name);
+}
+
+Status RangerAuthzProvider::FillTablePrivilegePB(const string& table_name,
+                                                 const string& user,
+                                                 const SchemaPB& schema_pb,
+                                                 TablePrivilegePB* pb) {
+  DCHECK(pb);
+  DCHECK(pb->has_table_id());
+  if (IsTrustedUser(user) || client_.AuthorizeAction(user, ActionPB::ALL, table_name).ok()) {
+    pb->set_delete_privilege(true);
+    pb->set_insert_privilege(true);
+    pb->set_scan_privilege(true);
+    pb->set_update_privilege(true);
+    return Status::OK();
+  }
+
+  unordered_set<ActionPB, ActionHash> actions = {
+    ActionPB::DELETE,
+    ActionPB::INSERT,
+    ActionPB::UPDATE,
+    ActionPB::SELECT
+  };
+
+  // Check if the user has any table-level privileges. If yes, we set them. If
+  // select is included, we can also return.
+  auto s = client_.AuthorizeActions(user, table_name, &actions);
+  if (s.ok()) {
+    for (const ActionPB& action : actions) {
+      switch (action) {
+        case ActionPB::DELETE:
+          pb->set_delete_privilege(true);
+          break;
+        case ActionPB::UPDATE:
+          pb->set_update_privilege(true);
+          break;
+        case ActionPB::INSERT:
+          pb->set_insert_privilege(true);
+          break;
+        case ActionPB::SELECT:
+          pb->set_scan_privilege(true);
+          break;
+        default:
+          LOG(WARNING) << "Unexpected action returned by Ranger: " << ActionPB_Name(action);
+          break;
+      }
+      if (pb->scan_privilege()) {
+        return Status::OK();
+      }
+    }
+  }
+
+  // If select is not allowed on the table level we need to dig in and set
+  // select permissions on the column level.
+  static ColumnPrivilegePB scan_col_privilege;
+  scan_col_privilege.set_scan_privilege(true);
+
+  unordered_set<string> column_names;
+  for (const auto& col : schema_pb.columns()) {
+    column_names.emplace(col.name());
+  }
+
+  // If AuthorizeAction returns NotAuthorized, that means no column-level select
+  // is allowed to the user. In this case we return the previous status.
+  // Otherwise we populate schema_pb and return OK.
+  //
+  // TODO(abukor): revisit if it's worth merge this into the previous request
+  if (!client_.AuthorizeActionMultipleColumns(user, ActionPB::SELECT, table_name,
+                                              &column_names).ok()) {
+    return s;
+  }
+
+  for (const auto& col : schema_pb.columns()) {
+    if (ContainsKey(column_names, col.name())) {
+      InsertOrDie(pb->mutable_column_privileges(), col.id(), scan_col_privilege);
+    }
+  }
+
+
+  return Status::OK();
+}
+
+bool RangerAuthzProvider::IsEnabled() {
+  return !FLAGS_ranger_config_path.empty();
+}
+
+} // namespace master
+} // namespace kudu
diff --git a/src/kudu/master/ranger_authz_provider.h b/src/kudu/master/ranger_authz_provider.h
new file mode 100644
index 0000000..bdfb11a
--- /dev/null
+++ b/src/kudu/master/ranger_authz_provider.h
@@ -0,0 +1,91 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#pragma once
+
+#include <string>
+#include <unordered_set>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/master/authz_provider.h"
+#include "kudu/ranger/ranger_client.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class MetricEntity;
+class SchemaPB;
+
+namespace security {
+class TablePrivilegePB;
+}  // namespace security
+
+namespace master {
+
+// An implementation of AuthzProvider that connects to Ranger and translates
+// authorization requests to Ranger and allows or denies the actions based on
+// the received responses.
+class RangerAuthzProvider : public AuthzProvider {
+ public:
+
+  explicit RangerAuthzProvider(const scoped_refptr<MetricEntity>& metric_entity);
+
+  Status Start() override;
+
+  void Stop() override {}
+
+  Status ResetCache() override {
+    return Status::NotSupported("Resetting cache is not supported with Ranger");
+  }
+
+  Status AuthorizeCreateTable(const std::string& table_name,
+                              const std::string& user,
+                              const std::string& owner) override WARN_UNUSED_RESULT;
+
+  Status AuthorizeDropTable(const std::string& table_name,
+                            const std::string& user) override WARN_UNUSED_RESULT;
+
+  Status AuthorizeAlterTable(const std::string& old_table,
+                             const std::string& new_table,
+                             const std::string& user) override WARN_UNUSED_RESULT;
+
+  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 AuthorizeGetTableStatistics(const std::string& table_name,
+                                     const std::string& user) override WARN_UNUSED_RESULT;
+
+  Status FillTablePrivilegePB(const std::string& table_name,
+                              const std::string& user,
+                              const SchemaPB& schema_pb,
+                              security::TablePrivilegePB* pb) override WARN_UNUSED_RESULT;
+
+  // Returns true if 'ranger_policy_server' flag is set indicating Ranger
+  // authorization is enabled.
+  static bool IsEnabled();
+
+ private:
+  ranger::RangerClient client_;
+};
+
+} // namespace master
+} // namespace kudu
diff --git a/src/kudu/master/sentry_authz_provider.cc b/src/kudu/master/sentry_authz_provider.cc
index b9705c0..7a0bb89 100644
--- a/src/kudu/master/sentry_authz_provider.cc
+++ b/src/kudu/master/sentry_authz_provider.cc
@@ -17,13 +17,13 @@
 
 #include "kudu/master/sentry_authz_provider.h"
 
+#include <algorithm>
 #include <unordered_map>
 #include <unordered_set>
 #include <utility>
 #include <vector>
 
 #include <gflags/gflags.h>
-#include <gflags/gflags_declare.h>
 #include <glog/logging.h>
 
 #include "kudu/common/common.pb.h"
@@ -42,7 +42,8 @@
 DEFINE_bool(sentry_require_db_privileges_for_list_tables, false,
             "Whether Kudu will require database-level privileges to authorize "
             "ListTables requests. When set to false, table-level privileges are "
-            "required for each table.");
+            "required for each table. ranger_config_path must not be set if "
+            "this is set");
 TAG_FLAG(sentry_require_db_privileges_for_list_tables, advanced);
 
 DECLARE_string(sentry_service_rpc_addresses);