diff --git a/src/auth/user_access_manager.cc b/src/auth/user_access_manager.cc index 32506d8cbcf..d668a7dc497 100644 --- a/src/auth/user_access_manager.cc +++ b/src/auth/user_access_manager.cc @@ -32,20 +32,17 @@ UserAccessManager::UserAccessManager(IteratorFactory iterator_factory) UserAccessManager::~UserAccessManager() { StopSyncTask(); } void UserAccessManager::StartSyncTask() { - sync_task_running_ = true; - sync_task_thread_ = std::thread([this] { - while (sync_task_running_) { + sync_task_thread_ = std::thread([this, fut = stop_promise_.get_future()] { + while (true) { SyncWithDB(); - std::this_thread::sleep_for(std::chrono::milliseconds(100)); + if (fut.wait_for(std::chrono::minutes(15)) != std::future_status::timeout) return; } }); } void UserAccessManager::StopSyncTask() { - sync_task_running_ = false; - if (sync_task_thread_.joinable()) { - sync_task_thread_.join(); - } + stop_promise_.set_value(); + sync_task_thread_.join(); } void UserAccessManager::SyncWithDB() { diff --git a/src/auth/user_access_manager.h b/src/auth/user_access_manager.h index af2dc0c6791..996efc326c4 100644 --- a/src/auth/user_access_manager.h +++ b/src/auth/user_access_manager.h @@ -19,6 +19,7 @@ #include #include +#include #include #include #include @@ -38,14 +39,13 @@ class UserAccessManager { ~UserAccessManager(); bool IsAuthenticated(const std::string& host, const std::string& username, const std::string& password); + void SyncWithDB(); private: IteratorFactory user_table_iterator_factory_; RefreshableMap user_map_; - std::atomic sync_task_running_{false}; std::thread sync_task_thread_; - - void SyncWithDB(); + std::promise stop_promise_; void StartSyncTask(); void StopSyncTask(); }; diff --git a/src/base/status.h b/src/base/status.h index 8ac134b18bd..c7e5ec75198 100644 --- a/src/base/status.h +++ b/src/base/status.h @@ -183,7 +183,10 @@ enum ReturnCode { kSQLRunError = 1001, kRPCRunError = 1002, kServerConnError = 1003, - kRPCError = 1004 // brpc controller error + kRPCError = 1004, // brpc controller error + + // auth + kFlushPrivilegesFailed = 1100 // brpc controller error }; struct Status { diff --git a/src/client/ns_client.cc b/src/client/ns_client.cc index 9a4baa549bc..cdeef07e521 100644 --- a/src/client/ns_client.cc +++ b/src/client/ns_client.cc @@ -19,6 +19,7 @@ #include #include "base/strings.h" +#include "ns_client.h" DECLARE_int32(request_timeout_ms); namespace openmldb { @@ -302,6 +303,33 @@ bool NsClient::CreateTable(const ::openmldb::nameserver::TableInfo& table_info, bool NsClient::DropTable(const std::string& name, std::string& msg) { return DropTable(GetDb(), name, msg); } +bool NsClient::PutUser(const std::string& host, const std::string& name, const std::string& password) { + ::openmldb::nameserver::PutUserRequest request; + request.set_host(host); + request.set_name(name); + request.set_password(password); + ::openmldb::nameserver::GeneralResponse response; + bool ok = client_.SendRequest(&::openmldb::nameserver::NameServer_Stub::PutUser, &request, &response, + FLAGS_request_timeout_ms, 1); + if (ok && response.code() == 0) { + return true; + } + return false; +} + +bool NsClient::DeleteUser(const std::string& host, const std::string& name) { + ::openmldb::nameserver::DeleteUserRequest request; + request.set_host(host); + request.set_name(name); + ::openmldb::nameserver::GeneralResponse response; + bool ok = client_.SendRequest(&::openmldb::nameserver::NameServer_Stub::DeleteUser, &request, &response, + FLAGS_request_timeout_ms, 1); + if (ok && response.code() == 0) { + return true; + } + return false; +} + bool NsClient::DropTable(const std::string& db, const std::string& name, std::string& msg) { ::openmldb::nameserver::DropTableRequest request; request.set_name(name); diff --git a/src/client/ns_client.h b/src/client/ns_client.h index 15a19f48ae7..73a52854765 100644 --- a/src/client/ns_client.h +++ b/src/client/ns_client.h @@ -110,6 +110,10 @@ class NsClient : public Client { bool DropTable(const std::string& name, std::string& msg); // NOLINT + bool PutUser(const std::string& host, const std::string& name, const std::string& password); // NOLINT + + bool DeleteUser(const std::string& host, const std::string& name); // NOLINT + bool DropTable(const std::string& db, const std::string& name, std::string& msg); // NOLINT diff --git a/src/client/tablet_client.cc b/src/client/tablet_client.cc index 09e7bdcbd96..a1dd925fcde 100644 --- a/src/client/tablet_client.cc +++ b/src/client/tablet_client.cc @@ -26,6 +26,7 @@ #include "codec/sql_rpc_row_codec.h" #include "common/timer.h" #include "sdk/sql_request_row.h" +#include "tablet_client.h" DECLARE_int32(request_max_retry); DECLARE_int32(request_timeout_ms); @@ -1414,5 +1415,16 @@ bool TabletClient::GetAndFlushDeployStats(::openmldb::api::DeployStatsResponse* return ok && res->code() == 0; } +bool TabletClient::FlushPrivileges() { + ::openmldb::api::EmptyRequest request; + ::openmldb::api::GeneralResponse response; + + bool ok = client_.SendRequest(&::openmldb::api::TabletServer_Stub::FlushPrivileges, &request, &response, + FLAGS_request_timeout_ms, 1); + if (ok && response.code() == 0) { + return true; + } + return false; +} } // namespace client } // namespace openmldb diff --git a/src/client/tablet_client.h b/src/client/tablet_client.h index 66155c968d7..177124208fc 100644 --- a/src/client/tablet_client.h +++ b/src/client/tablet_client.h @@ -267,6 +267,8 @@ class TabletClient : public Client { bool GetAndFlushDeployStats(::openmldb::api::DeployStatsResponse* res); + bool FlushPrivileges(); + private: base::Status LoadTableInternal(const ::openmldb::api::TableMeta& table_meta, std::shared_ptr task_info); diff --git a/src/cmd/openmldb.cc b/src/cmd/openmldb.cc index b13694d8d3c..3b3aa38cb5d 100644 --- a/src/cmd/openmldb.cc +++ b/src/cmd/openmldb.cc @@ -38,7 +38,6 @@ #endif #include "apiserver/api_server_impl.h" #include "auth/brpc_authenticator.h" -#include "auth/user_access_manager.h" #include "boost/algorithm/string.hpp" #include "boost/lexical_cast.hpp" #include "brpc/server.h" @@ -147,12 +146,10 @@ void StartNameServer() { } brpc::ServerOptions options; - std::unique_ptr user_access_manager; std::unique_ptr server_authenticator; - user_access_manager = std::make_unique(name_server->GetSystemTableIterator()); server_authenticator = std::make_unique( - [&user_access_manager](const std::string& host, const std::string& username, const std::string& password) { - return user_access_manager->IsAuthenticated(host, username, password); + [name_server](const std::string& host, const std::string& username, const std::string& password) { + return name_server->IsAuthenticated(host, username, password); }); options.auth = server_authenticator.get(); @@ -253,13 +250,11 @@ void StartTablet() { exit(1); } brpc::ServerOptions options; - std::unique_ptr user_access_manager; std::unique_ptr server_authenticator; - user_access_manager = std::make_unique(tablet->GetSystemTableIterator()); server_authenticator = std::make_unique( - [&user_access_manager](const std::string& host, const std::string& username, const std::string& password) { - return user_access_manager->IsAuthenticated(host, username, password); + [tablet](const std::string& host, const std::string& username, const std::string& password) { + return tablet->IsAuthenticated(host, username, password); }); options.auth = server_authenticator.get(); options.num_threads = FLAGS_thread_pool_size; diff --git a/src/cmd/sql_cmd_test.cc b/src/cmd/sql_cmd_test.cc index cedda42a6cd..fe8faa21504 100644 --- a/src/cmd/sql_cmd_test.cc +++ b/src/cmd/sql_cmd_test.cc @@ -245,7 +245,6 @@ TEST_P(DBSDKTest, TestUser) { ASSERT_TRUE(status.IsOK()); ASSERT_TRUE(true); auto opt = sr->GetRouterOptions(); - std::this_thread::sleep_for(std::chrono::seconds(1)); // TODO(oh2024): Remove when CREATE USER becomes strongly if (cs->IsClusterMode()) { auto real_opt = std::dynamic_pointer_cast(opt); sdk::SQLRouterOptions opt1; @@ -257,7 +256,6 @@ TEST_P(DBSDKTest, TestUser) { ASSERT_TRUE(router != nullptr); sr->ExecuteSQL(absl::StrCat("ALTER USER user1 SET OPTIONS(password='abc')"), &status); ASSERT_TRUE(status.IsOK()); - std::this_thread::sleep_for(std::chrono::seconds(1)); // TODO(oh2024): Remove when CREATE USER becomes strongly router = NewClusterSQLRouter(opt1); ASSERT_FALSE(router != nullptr); } else { @@ -271,7 +269,6 @@ TEST_P(DBSDKTest, TestUser) { ASSERT_TRUE(router != nullptr); sr->ExecuteSQL(absl::StrCat("ALTER USER user1 SET OPTIONS(password='abc')"), &status); ASSERT_TRUE(status.IsOK()); - std::this_thread::sleep_for(std::chrono::seconds(1)); // TODO(oh2024): Remove when CREATE USER becomes strongly router = NewStandaloneSQLRouter(opt1); ASSERT_FALSE(router != nullptr); } diff --git a/src/nameserver/name_server_impl.cc b/src/nameserver/name_server_impl.cc index 871adcb8d49..9c565272fb3 100644 --- a/src/nameserver/name_server_impl.cc +++ b/src/nameserver/name_server_impl.cc @@ -45,6 +45,7 @@ #include "boost/bind.hpp" #include "codec/row_codec.h" #include "gflags/gflags.h" +#include "name_server_impl.h" #include "schema/index_util.h" #include "schema/schema_adapter.h" @@ -522,7 +523,8 @@ NameServerImpl::NameServerImpl() thread_pool_(1), task_thread_pool_(FLAGS_name_server_task_pool_size), rand_(0xdeadbeef), - startup_mode_(::openmldb::type::StartupMode::kStandalone) {} + startup_mode_(::openmldb::type::StartupMode::kStandalone), + user_access_manager_(GetSystemTableIterator()) {} NameServerImpl::~NameServerImpl() { running_.store(false, std::memory_order_release); @@ -650,7 +652,7 @@ bool NameServerImpl::Recover() { if (!RecoverExternalFunction()) { return false; } - return true; + return FlushPrivileges().OK(); } bool NameServerImpl::RecoverExternalFunction() { @@ -1377,8 +1379,8 @@ void NameServerImpl::ShowTablet(RpcController* controller, const ShowTabletReque response->set_msg("ok"); } -base::Status NameServerImpl::InsertUserRecord(const std::string& host, const std::string& user, - const std::string& password) { +base::Status NameServerImpl::PutUserRecord(const std::string& host, const std::string& user, + const std::string& password) { std::shared_ptr table_info; if (!GetTableInfo(USER_INFO_NAME, INTERNAL_DB, &table_info)) { return {ReturnCode::kTableIsNotExist, "user table does not exist"}; @@ -1388,13 +1390,13 @@ base::Status NameServerImpl::InsertUserRecord(const std::string& host, const std row_values.push_back(host); row_values.push_back(user); row_values.push_back(password); - row_values.push_back(""); // password_last_changed - row_values.push_back(""); // password_expired_time - row_values.push_back(""); // create_time - row_values.push_back(""); // update_time - row_values.push_back(""); // account_type - row_values.push_back(""); // privileges - row_values.push_back(""); // extra_info + row_values.push_back("0"); // password_last_changed + row_values.push_back("0"); // password_expired_time + row_values.push_back("0"); // create_time + row_values.push_back("0"); // update_time + row_values.push_back("1"); // account_type + row_values.push_back("0"); // privileges + row_values.push_back("null"); // extra_info std::string encoded_row; codec::RowCodec::EncodeRow(row_values, table_info->column_desc(), 1, encoded_row); @@ -1410,11 +1412,56 @@ base::Status NameServerImpl::InsertUserRecord(const std::string& host, const std std::string endpoint = table_partition.partition_meta(meta_idx).endpoint(); auto table_ptr = GetTablet(endpoint); if (!table_ptr->client_->Put(tid, 0, cur_ts, encoded_row, dimensions).OK()) { - return {ReturnCode::kPutFailed, "failed to create initial user entry"}; + return {ReturnCode::kPutFailed, "failed to put user entry"}; } break; } } + return FlushPrivileges(); +} + +base::Status NameServerImpl::DeleteUserRecord(const std::string& host, const std::string& user) { + std::shared_ptr table_info; + if (!GetTableInfo(USER_INFO_NAME, INTERNAL_DB, &table_info)) { + return {ReturnCode::kTableIsNotExist, "user table does not exist"}; + } + uint32_t tid = table_info->tid(); + auto table_partition = table_info->table_partition(0); // only one partition for system table + std::string msg; + for (int meta_idx = 0; meta_idx < table_partition.partition_meta_size(); meta_idx++) { + if (table_partition.partition_meta(meta_idx).is_leader() && + table_partition.partition_meta(meta_idx).is_alive()) { + uint64_t cur_ts = ::baidu::common::timer::get_micros() / 1000; + std::string endpoint = table_partition.partition_meta(meta_idx).endpoint(); + auto table_ptr = GetTablet(endpoint); + if (!table_ptr->client_->Delete(tid, 0, host + "|" + user, "index", msg)) { + return {ReturnCode::kDeleteFailed, msg}; + } + + break; + } + } + return FlushPrivileges(); +} + +base::Status NameServerImpl::FlushPrivileges() { + user_access_manager_.SyncWithDB(); + std::vector failed_tablet_list; + { + std::lock_guard lock(mu_); + for (const auto& tablet_pair : tablets_) { + const std::shared_ptr& tablet_info = tablet_pair.second; + if (tablet_info && tablet_info->Health() && tablet_info->client_) { + if (!tablet_info->client_->FlushPrivileges()) { + failed_tablet_list.push_back(tablet_pair.first); + } + } + } + } + if (failed_tablet_list.size() > 0) { + return {ReturnCode::kFlushPrivilegesFailed, + "Failed to flush privileges to tablets: " + boost::algorithm::join(failed_tablet_list, ", ")}; + } return {}; } @@ -5593,7 +5640,7 @@ void NameServerImpl::OnLocked() { CreateDatabaseOrExit(INTERNAL_DB); if (db_table_info_[INTERNAL_DB].count(USER_INFO_NAME) == 0) { CreateSystemTableOrExit(SystemTableType::kUser); - InsertUserRecord("%", "root", "1e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855"); + PutUserRecord("%", "root", "1e3b0c44298fc1c149afbf4c8996fb92427ae41e4649b934ca495991b7852b855"); } if (IsClusterMode()) { if (tablets_.size() < FLAGS_system_table_replica_num) { @@ -9613,6 +9660,25 @@ NameServerImpl::GetSystemTableIterator() { }; } +void NameServerImpl::PutUser(RpcController* controller, const PutUserRequest* request, GeneralResponse* response, + Closure* done) { + brpc::ClosureGuard done_guard(done); + auto status = PutUserRecord(request->host(), request->name(), request->password()); + base::SetResponseStatus(status, response); +} + +void NameServerImpl::DeleteUser(RpcController* controller, const DeleteUserRequest* request, GeneralResponse* response, + Closure* done) { + brpc::ClosureGuard done_guard(done); + auto status = DeleteUserRecord(request->host(), request->name()); + base::SetResponseStatus(status, response); +} + +bool NameServerImpl::IsAuthenticated(const std::string& host, const std::string& username, + const std::string& password) { + return user_access_manager_.IsAuthenticated(host, username, password); +} + bool NameServerImpl::RecoverProcedureInfo() { db_table_sp_map_.clear(); db_sp_table_map_.clear(); diff --git a/src/nameserver/name_server_impl.h b/src/nameserver/name_server_impl.h index 9960fd3d247..dadc335c7a3 100644 --- a/src/nameserver/name_server_impl.h +++ b/src/nameserver/name_server_impl.h @@ -29,6 +29,7 @@ #include #include +#include "auth/user_access_manager.h" #include "base/hash.h" #include "base/random.h" #include "catalog/distribute_iterator.h" @@ -358,15 +359,23 @@ class NameServerImpl : public NameServer { void DropProcedure(RpcController* controller, const api::DropProcedureRequest* request, GeneralResponse* response, Closure* done); + void PutUser(RpcController* controller, const PutUserRequest* request, GeneralResponse* response, Closure* done); + void DeleteUser(RpcController* controller, const DeleteUserRequest* request, GeneralResponse* response, + Closure* done); + bool IsAuthenticated(const std::string& host, const std::string& username, const std::string& password); + + private: + std::function, std::unique_ptr>>(const std::string& table_name)> GetSystemTableIterator(); - + bool GetTableInfo(const std::string& table_name, const std::string& db_name, std::shared_ptr* table_info); - private: - base::Status InsertUserRecord(const std::string& host, const std::string& user, const std::string& password); + base::Status PutUserRecord(const std::string& host, const std::string& user, const std::string& password); + base::Status DeleteUserRecord(const std::string& host, const std::string& user); + base::Status FlushPrivileges(); base::Status InitGlobalVarTable(); @@ -735,6 +744,7 @@ class NameServerImpl : public NameServer { std::unordered_map>> db_sp_info_map_; ::openmldb::type::StartupMode startup_mode_; + openmldb::auth::UserAccessManager user_access_manager_; }; } // namespace nameserver diff --git a/src/proto/name_server.proto b/src/proto/name_server.proto index c75dca8f5a9..f7c8fd5c830 100755 --- a/src/proto/name_server.proto +++ b/src/proto/name_server.proto @@ -533,6 +533,17 @@ message TableIndex { repeated openmldb.common.ColumnKey column_key = 3; } +message PutUserRequest { + required string host = 1; + required string name = 2; + required string password = 3; +} + +message DeleteUserRequest { + required string host = 1; + required string name = 2; +} + message DeploySQLRequest { optional openmldb.api.ProcedureInfo sp_info = 3; repeated TableIndex index = 4; @@ -602,4 +613,8 @@ service NameServer { rpc DropProcedure(openmldb.api.DropProcedureRequest) returns (GeneralResponse); rpc ShowProcedure(openmldb.api.ShowProcedureRequest) returns (openmldb.api.ShowProcedureResponse); rpc DeploySQL(DeploySQLRequest) returns (DeploySQLResponse); + + // user related interfaces + rpc PutUser(PutUserRequest) returns (GeneralResponse); + rpc DeleteUser(DeleteUserRequest) returns (GeneralResponse); } diff --git a/src/proto/tablet.proto b/src/proto/tablet.proto index a1ae6e72d5a..bc160a01f1e 100755 --- a/src/proto/tablet.proto +++ b/src/proto/tablet.proto @@ -977,4 +977,7 @@ service TabletServer { rpc CreateAggregator(CreateAggregatorRequest) returns (CreateAggregatorResponse); // monitoring interfaces rpc GetAndFlushDeployStats(GAFDeployStatsRequest) returns (DeployStatsResponse); + + // flush privilege + rpc FlushPrivileges(EmptyRequest) returns (GeneralResponse); } diff --git a/src/sdk/mini_cluster.h b/src/sdk/mini_cluster.h index 673e1cb1f61..24521005772 100644 --- a/src/sdk/mini_cluster.h +++ b/src/sdk/mini_cluster.h @@ -26,7 +26,6 @@ #include #include "auth/brpc_authenticator.h" -#include "auth/user_access_manager.h" #include "base/file_util.h" #include "base/glog_wrapper.h" #include "brpc/server.h" @@ -74,12 +73,6 @@ class MiniCluster { } ~MiniCluster() { - for (auto& tablet_user_access_manager : tablet_user_access_managers_) { - if (tablet_user_access_manager) { - delete tablet_user_access_manager; - tablet_user_access_manager = nullptr; - } - } for (auto& tablet_authenticator : tablet_authenticators_) { if (tablet_authenticator) { delete tablet_authenticator; @@ -87,11 +80,6 @@ class MiniCluster { } } - if (user_access_manager_) { - delete user_access_manager_; - user_access_manager_ = nullptr; - } - if (ns_authenticator_) { delete ns_authenticator_; ns_authenticator_ = nullptr; @@ -137,15 +125,9 @@ class MiniCluster { if (!ok) { return false; } - if (!nameserver->GetTableInfo(::openmldb::nameserver::USER_INFO_NAME, ::openmldb::nameserver::INTERNAL_DB, - &user_table_info_)) { - PDLOG(WARNING, "Failed to get table info for user table"); - return false; - } - user_access_manager_ = new openmldb::auth::UserAccessManager(nameserver->GetSystemTableIterator()); ns_authenticator_ = new openmldb::authn::BRPCAuthenticator( [this](const std::string& host, const std::string& username, const std::string& password) { - return user_access_manager_->IsAuthenticated(host, username, password); + return nameserver->IsAuthenticated(host, username, password); }); brpc::ServerOptions options; options.auth = ns_authenticator_; @@ -173,23 +155,12 @@ class MiniCluster { } void Close() { - for (auto& tablet_user_access_manager : tablet_user_access_managers_) { - if (tablet_user_access_manager) { - delete tablet_user_access_manager; - tablet_user_access_manager = nullptr; - } - } for (auto& tablet_authenticator : tablet_authenticators_) { if (tablet_authenticator) { delete tablet_authenticator; tablet_authenticator = nullptr; } } - if (user_access_manager_) { - delete user_access_manager_; - user_access_manager_ = nullptr; - } - if (ns_authenticator_) { delete ns_authenticator_; ns_authenticator_ = nullptr; @@ -244,13 +215,10 @@ class MiniCluster { return false; } - auto tablet_user_access_manager = new openmldb::auth::UserAccessManager(tablet->GetSystemTableIterator()); auto ts_authenticator = new openmldb::authn::BRPCAuthenticator( - [tablet_user_access_manager](const std::string& host, const std::string& username, - const std::string& password) { - return tablet_user_access_manager->IsAuthenticated(host, username, password); + [tablet](const std::string& host, const std::string& username, const std::string& password) { + return tablet->IsAuthenticated(host, username, password); }); - tablet_user_access_managers_.push_back(tablet_user_access_manager); tablet_authenticators_.push_back(ts_authenticator); brpc::ServerOptions options; options.auth = ts_authenticator; @@ -291,22 +259,13 @@ class MiniCluster { std::map tablets_; std::map tb_clients_; openmldb::authn::BRPCAuthenticator* ns_authenticator_; - openmldb::auth::UserAccessManager* user_access_manager_; - std::vector tablet_user_access_managers_; std::vector tablet_authenticators_; - std::shared_ptr<::openmldb::nameserver::TableInfo> user_table_info_; }; class StandaloneEnv { public: StandaloneEnv() : ns_(), ns_client_(nullptr), tb_client_(nullptr) { FLAGS_skip_grant_tables = false; } ~StandaloneEnv() { - for (auto& tablet_user_access_manager : tablet_user_access_managers_) { - if (tablet_user_access_manager) { - delete tablet_user_access_manager; - tablet_user_access_manager = nullptr; - } - } for (auto& tablet_authenticator : tablet_authenticators_) { if (tablet_authenticator) { delete tablet_authenticator; @@ -314,11 +273,6 @@ class StandaloneEnv { } } - if (user_access_manager_) { - delete user_access_manager_; - user_access_manager_ = nullptr; - } - if (ns_authenticator_) { delete ns_authenticator_; ns_authenticator_ = nullptr; @@ -353,15 +307,9 @@ class StandaloneEnv { if (!ok) { return false; } - if (!nameserver->GetTableInfo(::openmldb::nameserver::USER_INFO_NAME, ::openmldb::nameserver::INTERNAL_DB, - &user_table_info_)) { - PDLOG(WARNING, "Failed to get table info for user table"); - return false; - } - user_access_manager_ = new openmldb::auth::UserAccessManager(nameserver->GetSystemTableIterator()); ns_authenticator_ = new openmldb::authn::BRPCAuthenticator( [this](const std::string& host, const std::string& username, const std::string& password) { - return user_access_manager_->IsAuthenticated(host, username, password); + return nameserver->IsAuthenticated(host, username, password); }); brpc::ServerOptions options; options.auth = ns_authenticator_; @@ -387,12 +335,6 @@ class StandaloneEnv { } void Close() { - for (auto& tablet_user_access_manager : tablet_user_access_managers_) { - if (tablet_user_access_manager) { - delete tablet_user_access_manager; - tablet_user_access_manager = nullptr; - } - } for (auto& tablet_authenticator : tablet_authenticators_) { if (tablet_authenticator) { delete tablet_authenticator; @@ -400,11 +342,6 @@ class StandaloneEnv { } } - if (user_access_manager_) { - delete user_access_manager_; - user_access_manager_ = nullptr; - } - if (ns_authenticator_) { delete ns_authenticator_; ns_authenticator_ = nullptr; @@ -436,15 +373,12 @@ class StandaloneEnv { bool ok = tablet->Init("", "", tb_endpoint, ""); if (!ok) { return false; - } + } - auto tablet_user_access_manager = new openmldb::auth::UserAccessManager(tablet->GetSystemTableIterator()); auto ts_authenticator = new openmldb::authn::BRPCAuthenticator( - [tablet_user_access_manager](const std::string& host, const std::string& username, - const std::string& password) { - return tablet_user_access_manager->IsAuthenticated(host, username, password); + [tablet](const std::string& host, const std::string& username, const std::string& password) { + return tablet->IsAuthenticated(host, username, password); }); - tablet_user_access_managers_.push_back(tablet_user_access_manager); tablet_authenticators_.push_back(ts_authenticator); brpc::ServerOptions options; options.auth = ts_authenticator; @@ -474,9 +408,6 @@ class StandaloneEnv { ::openmldb::client::NsClient* ns_client_; ::openmldb::client::TabletClient* tb_client_; openmldb::authn::BRPCAuthenticator* ns_authenticator_; - openmldb::auth::UserAccessManager* user_access_manager_; - std::shared_ptr<::openmldb::nameserver::TableInfo> user_table_info_; - std::vector tablet_user_access_managers_; std::vector tablet_authenticators_; }; diff --git a/src/sdk/sql_cluster_router.cc b/src/sdk/sql_cluster_router.cc index 705fbd62400..e58eb8cd2cc 100644 --- a/src/sdk/sql_cluster_router.cc +++ b/src/sdk/sql_cluster_router.cc @@ -4902,49 +4902,51 @@ absl::StatusOr SQLClusterRouter::GetUser(const std::string& name, UserInfo hybridse::sdk::Status SQLClusterRouter::AddUser(const std::string& name, const std::string& password) { auto real_password = password.empty() ? password : codec::Encrypt(password); - uint64_t cur_ts = ::baidu::common::timer::get_micros() / 1000; - std::string sql = absl::StrCat("insert into ", nameserver::USER_INFO_NAME, " values (", - "'%',", // host - "'", name, "','", // user - real_password, "',", // password - cur_ts, ",", // password_last_changed - "0,", // password_expired_time - cur_ts, ", ", // create_time - cur_ts, ",", // update_time - 1, // account_type - ",'',", // privileges - "null" // extra_info - ");"); + hybridse::sdk::Status status; - ExecuteInsert(nameserver::INTERNAL_DB, sql, &status); + + auto ns_client = cluster_sdk_->GetNsClient(); + + bool ok = ns_client->PutUser("%", name, real_password); + + if (!ok) { + status.code = hybridse::common::StatusCode::kRunError; + status.msg = absl::StrCat("Fail to create user: ", name); + } + return status; } hybridse::sdk::Status SQLClusterRouter::UpdateUser(const UserInfo& user_info, const std::string& password) { + auto name = user_info.name; auto real_password = password.empty() ? password : codec::Encrypt(password); - uint64_t cur_ts = ::baidu::common::timer::get_micros() / 1000; - std::string sql = absl::StrCat("insert into ", nameserver::USER_INFO_NAME, " values (", - "'%',", // host - "'", user_info.name, "','", // user - real_password, "',", // password - cur_ts, ",", // password_last_changed - "0,", // password_expired_time - user_info.create_time, ", ", // create_time - cur_ts, ",", // update_time - 1, // account_type - ",'", user_info.privileges, "',", // privileges - "null" // extra_info - ");"); + hybridse::sdk::Status status; - ExecuteInsert(nameserver::INTERNAL_DB, sql, &status); + + auto ns_client = cluster_sdk_->GetNsClient(); + + bool ok = ns_client->PutUser("%", name, real_password); + + if (!ok) { + status.code = hybridse::common::StatusCode::kRunError; + status.msg = absl::StrCat("Fail to update user: ", name); + } + return status; } hybridse::sdk::Status SQLClusterRouter::DeleteUser(const std::string& name) { - std::string sql = absl::StrCat("delete from ", nameserver::USER_INFO_NAME, - " where host = '%' and user = '", name, "';"); hybridse::sdk::Status status; - ExecuteSQL(nameserver::INTERNAL_DB, sql, &status); + + auto ns_client = cluster_sdk_->GetNsClient(); + + bool ok = ns_client->DeleteUser("%", name); + + if (!ok) { + status.code = hybridse::common::StatusCode::kRunError; + status.msg = absl::StrCat("Fail to delete user: ", name); + } + return status; } diff --git a/src/tablet/tablet_impl.cc b/src/tablet/tablet_impl.cc index 8c59a4f9184..2f7544f2847 100644 --- a/src/tablet/tablet_impl.cc +++ b/src/tablet/tablet_impl.cc @@ -154,7 +154,8 @@ TabletImpl::TabletImpl() sp_cache_(std::shared_ptr(new SpCache())), notify_path_(), globalvar_changed_notify_path_(), - startup_mode_(::openmldb::type::StartupMode::kStandalone) {} + startup_mode_(::openmldb::type::StartupMode::kStandalone), + user_access_manager_(GetSystemTableIterator()) {} TabletImpl::~TabletImpl() { task_pool_.Stop(true); @@ -5814,6 +5815,17 @@ void TabletImpl::GetAndFlushDeployStats(::google::protobuf::RpcController* contr response->set_code(ReturnCode::kOk); } +bool TabletImpl::IsAuthenticated(const std::string& host, const std::string& username, const std::string& password) { + return user_access_manager_.IsAuthenticated(host, username, password); +} + +void TabletImpl::FlushPrivileges(::google::protobuf::RpcController* controller, + const ::openmldb::api::EmptyRequest* request, + ::openmldb::api::GeneralResponse* response, ::google::protobuf::Closure* done) { + brpc::ClosureGuard done_guard(done); + user_access_manager_.SyncWithDB(); +} + std::function, std::unique_ptr>>(const std::string& table_name)> TabletImpl::GetSystemTableIterator() { diff --git a/src/tablet/tablet_impl.h b/src/tablet/tablet_impl.h index 89ab1c1befa..d299956cb9e 100644 --- a/src/tablet/tablet_impl.h +++ b/src/tablet/tablet_impl.h @@ -26,6 +26,7 @@ #include #include +#include "auth/user_access_manager.h" #include "base/spinlock.h" #include "brpc/server.h" #include "catalog/tablet_catalog.h" @@ -274,11 +275,15 @@ class TabletImpl : public ::openmldb::api::TabletServer { ::openmldb::api::DeployStatsResponse* response, ::google::protobuf::Closure* done) override; + bool IsAuthenticated(const std::string& host, const std::string& username, const std::string& password); + void FlushPrivileges(::google::protobuf::RpcController* controller, const ::openmldb::api::EmptyRequest* request, + ::openmldb::api::GeneralResponse* response, ::google::protobuf::Closure* done); + + private: std::function, std::unique_ptr>>(const std::string& table_name)> GetSystemTableIterator(); - private: class UpdateAggrClosure : public Closure { public: explicit UpdateAggrClosure(const std::function& callback) : callback_(callback) {} @@ -489,6 +494,7 @@ class TabletImpl : public ::openmldb::api::TabletServer { std::unique_ptr deploy_collector_; std::atomic memory_used_ = 0; std::atomic system_memory_usage_rate_ = 0; // [0, 100] + openmldb::auth::UserAccessManager user_access_manager_; }; } // namespace tablet