diff --git a/programs/client/clickhouse-client.xml b/programs/client/clickhouse-client.xml index ae523154f8d..66e7afd8f8c 100644 --- a/programs/client/clickhouse-client.xml +++ b/programs/client/clickhouse-client.xml @@ -29,6 +29,7 @@ {display_name} \x01\e[1;32m\x02:)\x01\e[0m\x02 {display_name} \x01\e[1;31m\x02:)\x01\e[0m\x02 + 3 - 0 + - 9100 + false @@ -243,7 +243,7 @@ openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096 Only file format with BEGIN DH PARAMETERS is supported. --> - + none true true @@ -367,6 +367,9 @@ /var/lib/clickhouse/tmp/ + + + - true - true + false false diff --git a/programs/server/users.xml b/programs/server/users.xml index ab286a5a83b..fd5fe414579 100644 --- a/programs/server/users.xml +++ b/programs/server/users.xml @@ -29,7 +29,7 @@ - + - + --> + + default + - default + default diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index ed211df2e58..e2636286670 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -172,7 +172,7 @@ void AccessControl::addUsersConfigStorage(const String & storage_name_, const Po { auto check_setting_name_function = [this](const std::string_view & setting_name) { checkSettingNameIsAllowed(setting_name); }; auto new_storage = std::make_shared(storage_name_, check_setting_name_function); - new_storage->setConfig(users_config_); + new_storage->setConfig(users_config_, allow_plaintext_password); addStorage(new_storage); LOG_DEBUG(getLogger(), "Added {} access storage '{}', path: {}", String(new_storage->getStorageType()), new_storage->getStorageName(), new_storage->getPath()); @@ -182,10 +182,10 @@ void AccessControl::addUsersConfigStorage( const String & users_config_path_, const String & include_from_path_, const String & preprocessed_dir_, - const zkutil::GetZooKeeper & get_zookeeper_function_, const bool allow_plaintext_password) + const zkutil::GetZooKeeper & get_zookeeper_function_) { addUsersConfigStorage( - UsersConfigAccessStorage::STORAGE_TYPE, users_config_path_, include_from_path_, preprocessed_dir_, get_zookeeper_function_,allow_plaintext_password); + UsersConfigAccessStorage::STORAGE_TYPE, users_config_path_, include_from_path_, preprocessed_dir_, get_zookeeper_function_); } void AccessControl::addUsersConfigStorage( @@ -193,7 +193,7 @@ void AccessControl::addUsersConfigStorage( const String & users_config_path_, const String & include_from_path_, const String & preprocessed_dir_, - const zkutil::GetZooKeeper & get_zookeeper_function_,const bool allow_plaintext_password) + const zkutil::GetZooKeeper & get_zookeeper_function_) { auto storages = getStoragesPtr(); for (const auto & storage : *storages) @@ -206,11 +206,14 @@ void AccessControl::addUsersConfigStorage( } auto check_setting_name_function = [this](const std::string_view & setting_name) { checkSettingNameIsAllowed(setting_name); }; auto new_storage = std::make_shared(storage_name_, check_setting_name_function); - new_storage->load(users_config_path_, include_from_path_, preprocessed_dir_, get_zookeeper_function_,allow_plaintext_password); + new_storage->load(users_config_path_, include_from_path_, preprocessed_dir_, get_zookeeper_function_, allow_plaintext_password); addStorage(new_storage); LOG_DEBUG(getLogger(), "Added {} access storage '{}', path: {}", String(new_storage->getStorageType()), new_storage->getStorageName(), new_storage->getPath()); } - +void AccessControl::setAllowPlaintextPasswordSetting(const bool allow_plaintext_password_) +{ + allow_plaintext_password = allow_plaintext_password_; +} void AccessControl::reloadUsersConfigs() { auto storages = getStoragesPtr(); @@ -332,7 +335,6 @@ void AccessControl::addStoragesFromUserDirectoriesConfig( type = LDAPAccessStorage::STORAGE_TYPE; String name = config.getString(prefix + ".name", type); - const bool allow_plaintext_password = config.getBool("enable_plaintext_password" ,0); if (type == MemoryAccessStorage::STORAGE_TYPE) { @@ -343,7 +345,7 @@ void AccessControl::addStoragesFromUserDirectoriesConfig( String path = config.getString(prefix + ".path"); if (std::filesystem::path{path}.is_relative() && std::filesystem::exists(config_dir + path)) path = config_dir + path; - addUsersConfigStorage(name, path, include_from_path, dbms_dir, get_zookeeper_function,allow_plaintext_password); + addUsersConfigStorage(name, path, include_from_path, dbms_dir, get_zookeeper_function); } else if (type == DiskAccessStorage::STORAGE_TYPE) { diff --git a/src/Access/AccessControl.h b/src/Access/AccessControl.h index de32938e52f..7763c2b45c4 100644 --- a/src/Access/AccessControl.h +++ b/src/Access/AccessControl.h @@ -61,13 +61,13 @@ public: void addUsersConfigStorage(const String & users_config_path_, const String & include_from_path_, const String & preprocessed_dir_, - const zkutil::GetZooKeeper & get_zookeeper_function_ = {},const bool allow_plaintext_password=0); + const zkutil::GetZooKeeper & get_zookeeper_function_ = {}); void addUsersConfigStorage(const String & storage_name_, const String & users_config_path_, const String & include_from_path_, const String & preprocessed_dir_, - const zkutil::GetZooKeeper & get_zookeeper_function_ = {},const bool allow_plaintext_password=0); + const zkutil::GetZooKeeper & get_zookeeper_function_ = {}); void reloadUsersConfigs(); void startPeriodicReloadingUsersConfigs(); @@ -144,6 +144,9 @@ public: const String & forwarded_address, const String & custom_quota_key) const; + void setAllowPlaintextPasswordSetting(const bool allow_plaintext_password); + bool getAllowPlaintextPasswordSetting() const { return allow_plaintext_password; } + std::vector getAllQuotasUsage() const; std::shared_ptr getEnabledSettings( @@ -167,6 +170,7 @@ private: std::unique_ptr settings_profiles_cache; std::unique_ptr external_authenticators; std::unique_ptr custom_settings_prefixes; + bool allow_plaintext_password; }; } diff --git a/src/Access/Common/AuthenticationData.cpp b/src/Access/Common/AuthenticationData.cpp index 2d43c6a7340..7412d7336e3 100644 --- a/src/Access/Common/AuthenticationData.cpp +++ b/src/Access/Common/AuthenticationData.cpp @@ -95,6 +95,7 @@ bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs) && (lhs.ldap_server_name == rhs.ldap_server_name) && (lhs.kerberos_realm == rhs.kerberos_realm); } + void AuthenticationData::setPassword(const String & password_) { switch (type) @@ -127,10 +128,6 @@ String AuthenticationData::getPassword() const return String(password_hash.data(), password_hash.data() + password_hash.size()); } -void AuthenticationData::setPlaintextPasswordSetting(const bool allow_plaintext_password_ ) -{ - allow_plaintext_password = allow_plaintext_password_; -} void AuthenticationData::setPasswordHashHex(const String & hash) { diff --git a/src/Access/Common/AuthenticationData.h b/src/Access/Common/AuthenticationData.h index e36995d5e7f..8b390fd0900 100644 --- a/src/Access/Common/AuthenticationData.h +++ b/src/Access/Common/AuthenticationData.h @@ -60,13 +60,6 @@ public: /// Sets the password and encrypt it using the authentication type set in the constructor. void setPassword(const String & password_); - ///sets the config enable_plaintext_password value from global context - void setPlaintextPasswordSetting(const bool allow_plaintext_password_); - bool getPlaintextPasswordSetting() const { - std::cout<<"Heena - Inside this function"<< allow_plaintext_password <<"\n"; - return allow_plaintext_password; - } - /// Returns the password. Allowed to use only for Type::PLAINTEXT_PASSWORD. String getPassword() const; @@ -104,7 +97,6 @@ private: Digest password_hash; String ldap_server_name; String kerberos_realm; - bool allow_plaintext_password; }; } diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 76638b80097..f0a13618096 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -19,9 +19,6 @@ #include #include #include -#include "Poco/Logger.h" -#include - namespace DB @@ -31,8 +28,9 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int UNKNOWN_ADDRESS_PATTERN_TYPE; extern const int NOT_IMPLEMENTED; -} + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} namespace { @@ -73,35 +71,24 @@ namespace if (num_password_fields < 1) throw Exception("Either 'password' or 'password_sha256_hex' or 'password_double_sha1_hex' or 'no_password' or 'ldap' or 'kerberos' must be specified for user " + user_name + ".", ErrorCodes::BAD_ARGUMENTS); - //bool user_has_allow_plaintext_password = config.has(user_config + ".enable_plaintext_password"); - // bool user_allow_plaintext_password=0 ; - // if(user_has_allow_plaintext_password) - // bool allow_plaintext_password = config.getBool(user_config + ".enable_plaintext_password",allow_plaintext_password_); - - bool allow_plaintext_password = allow_plaintext_password_; - - if (has_password_plaintext) + bool allow_plaintext_password = allow_plaintext_password_; + if (has_password_plaintext) { - if(!allow_plaintext_password) - // LOG_ERROR(log(&Poco::Logger::get("BaseDaemon")), "Incorrect User configuration. User is not allowed to configure PLAINTEXT_PASSWORD if the setting name enable_plaintext_password setting is off"); - // LOG_ERROR(log(&Poco::Logger::get("BaseDaemon"), "Incorrect User configuration. User is not allowed to configure PLAINTEXT_PASSWORD if the setting name enable_plaintext_password setting is off.Please chnage the authtype to SHA256_PASSWORD,DOUBLE_SHA1_PASSWORD"); - throw Exception("Incorrect User configuration. User is not allowed to configure PLAINTEXT_PASSWORD. Please configure User with authtype to SHA256_PASSWORD,DOUBLE_SHA1_PASSWORD OR enable setting enable_plaintext_password in server configuration to configure user with plaintext password" - "It is recommended to use plaintext_password.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (!allow_plaintext_password) + throw Exception("Incorrect User configuration. User is not allowed to configure PLAINTEXT_PASSWORD. Please configure User with authtype SHA256_PASSWORD,DOUBLE_SHA1_PASSWORD OR enable setting allow_plaintext_password in server configuration to configure user with plaintext password" + " Though it is not recommended to use plaintext_password.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); user->auth_data = AuthenticationData{AuthenticationType::PLAINTEXT_PASSWORD}; user->auth_data.setPassword(config.getString(user_config + ".password")); - user->auth_data.setPlaintextPasswordSetting(allow_plaintext_password); } else if (has_password_sha256_hex) { user->auth_data = AuthenticationData{AuthenticationType::SHA256_PASSWORD}; user->auth_data.setPasswordHashHex(config.getString(user_config + ".password_sha256_hex")); - user->auth_data.setPlaintextPasswordSetting(allow_plaintext_password); } else if (has_password_double_sha1_hex) { user->auth_data = AuthenticationData{AuthenticationType::DOUBLE_SHA1_PASSWORD}; user->auth_data.setPasswordHashHex(config.getString(user_config + ".password_double_sha1_hex")); - user->auth_data.setPlaintextPasswordSetting(allow_plaintext_password); } else if (has_ldap) { @@ -219,7 +206,7 @@ namespace } - std::vector parseUsers(const Poco::Util::AbstractConfiguration & config,const bool allow_plaintext_password) + std::vector parseUsers(const Poco::Util::AbstractConfiguration & config, const bool allow_plaintext_password) { Poco::Util::AbstractConfiguration::Keys user_names; config.keys("users", user_names); @@ -231,7 +218,7 @@ namespace { try { - users.push_back(parseUser(config, user_name,allow_plaintext_password)); + users.push_back(parseUser(config, user_name, allow_plaintext_password)); } catch (Exception & e) { @@ -542,12 +529,12 @@ bool UsersConfigAccessStorage::isPathEqual(const String & path_) const } -void UsersConfigAccessStorage::setConfig(const Poco::Util::AbstractConfiguration & config) +void UsersConfigAccessStorage::setConfig(const Poco::Util::AbstractConfiguration & config, const bool allow_plaintext_password) { std::lock_guard lock{load_mutex}; path.clear(); config_reloader.reset(); - parseFromConfig(config,allow_plaintext_password); + parseFromConfig(config, allow_plaintext_password); } void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfiguration & config, const bool allow_plaintext_password) @@ -555,7 +542,7 @@ void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfigu try { std::vector> all_entities; - for (const auto & entity : parseUsers(config,allow_plaintext_password)) + for (const auto & entity : parseUsers(config, allow_plaintext_password)) all_entities.emplace_back(generateID(*entity), entity); for (const auto & entity : parseQuotas(config)) all_entities.emplace_back(generateID(*entity), entity); @@ -576,12 +563,10 @@ void UsersConfigAccessStorage::load( const String & users_config_path, const String & include_from_path, const String & preprocessed_dir, - const zkutil::GetZooKeeper & get_zookeeper_function, const bool allow_plaintext_password_flag) + const zkutil::GetZooKeeper & get_zookeeper_function, const bool allow_plaintext_password_) { - // allow_plaintext_password = allow_plaintext_password; std::lock_guard lock{load_mutex}; path = std::filesystem::path{users_config_path}.lexically_normal(); - allow_plaintext_password = allow_plaintext_password_flag; config_reloader.reset(); config_reloader = std::make_unique( users_config_path, @@ -591,7 +576,7 @@ void UsersConfigAccessStorage::load( std::make_shared(), [&](Poco::AutoPtr new_config, bool /*initial_loading*/) { - parseFromConfig(*new_config,allow_plaintext_password); + parseFromConfig(*new_config, allow_plaintext_password_); Settings::checkNoSettingNamesAtTopLevel(*new_config, users_config_path); }, diff --git a/src/Access/UsersConfigAccessStorage.h b/src/Access/UsersConfigAccessStorage.h index 060066c0df3..58f1b4899cf 100644 --- a/src/Access/UsersConfigAccessStorage.h +++ b/src/Access/UsersConfigAccessStorage.h @@ -32,12 +32,12 @@ public: String getPath() const; bool isPathEqual(const String & path_) const; - void setConfig(const Poco::Util::AbstractConfiguration & config); + void setConfig(const Poco::Util::AbstractConfiguration & config, const bool allow_plaintext_password=1); void load(const String & users_config_path, const String & include_from_path = {}, const String & preprocessed_dir = {}, - const zkutil::GetZooKeeper & get_zookeeper_function = {},const bool allow_plaintext_password=0); + const zkutil::GetZooKeeper & get_zookeeper_function = {}, const bool allow_plaintext_password=1); void reload(); void startPeriodicReloading(); void stopPeriodicReloading(); @@ -47,7 +47,7 @@ public: bool hasSubscription(AccessEntityType type) const override; private: - void parseFromConfig(const Poco::Util::AbstractConfiguration & config,const bool allow_plaintext_password); + void parseFromConfig(const Poco::Util::AbstractConfiguration & config, const bool allow_plaintext_password); std::optional findImpl(AccessEntityType type, const String & name) const override; std::vector findAllImpl(AccessEntityType type) const override; @@ -61,6 +61,5 @@ private: String path; std::unique_ptr config_reloader; mutable std::mutex load_mutex; - bool allow_plaintext_password; }; } diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index fd9133f024d..7970dba5e8d 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -14,12 +14,11 @@ namespace DB { - namespace ErrorCodes { - extern const int LOGICAL_ERROR; -} + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} namespace { void updateUserFromQueryImpl( @@ -28,7 +27,7 @@ namespace const std::shared_ptr & override_name, const std::optional & override_default_roles, const std::optional & override_settings, - const std::optional & override_grantees,const bool allow_plaintext_password) + const std::optional & override_grantees, const bool allow_plaintext_password) { if (override_name) user.setName(override_name->toString()); @@ -37,15 +36,14 @@ namespace else if (query.names->size() == 1) user.setName(query.names->front()->toString()); - if (query.auth_data){ - if(query.auth_data->getType()==AuthenticationType::PLAINTEXT_PASSWORD && !allow_plaintext_password) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "InterPreterQuery: User is not allowed to Create users with type "+ toString(query.auth_data->getType())+"Please configure User with authtype " - "to SHA256_PASSWORD,DOUBLE_SHA1_PASSWORD OR enable setting enable_plaintext_password in server configuration to configure user with plaintext password" - "It is recommended to use plaintext_password."); - // throw Exception("InterPreterQuery: User is not allowed to Create users with type " + toString(query.auth_data->getType()), ErrorCodes::LOGICAL_ERROR); + if (query.auth_data) + { + if (query.auth_data->getType() == AuthenticationType::PLAINTEXT_PASSWORD && !allow_plaintext_password) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "InterPreterQuery: User is not allowed to Create users with type "+ toString(query.auth_data->getType())+"Please configure User with authtype " + "to SHA256_PASSWORD,DOUBLE_SHA1_PASSWORD OR enable setting allow_plaintext_password in server configuration to configure user with plaintext password" + "Though it is recommended to use plaintext_password."); user.auth_data = *query.auth_data; } - if (override_name && !override_name->host_pattern.empty()) { user.allowed_client_hosts = AllowedClientHosts{}; @@ -87,22 +85,13 @@ namespace } } - - BlockIO InterpreterCreateUserQuery::execute() { const auto & query = query_ptr->as(); auto & access_control = getContext()->getAccessControl(); auto access = getContext()->getAccess(); access->checkAccess(query.alter ? AccessType::ALTER_USER : AccessType::CREATE_USER); - String username= access->getUserName(); - std::cout<<"Heena - lets see what is the user name: "<< access->getUserName(); - auto user = access->getUser(); - bool allow_plaintext_password = user->auth_data.getPlaintextPasswordSetting(); - // bool allow_plaintext_password = getContext().getSettingsRef().allow_plaintext_password - - if(allow_plaintext_password) - std::cout<<"Heena - Finally got"; + bool allow_plaintext_password = access_control.getAllowPlaintextPasswordSetting(); std::optional default_roles_from_query; if (query.default_roles) @@ -131,7 +120,7 @@ BlockIO InterpreterCreateUserQuery::execute() auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr { auto updated_user = typeid_cast>(entity->clone()); - updateUserFromQueryImpl(*updated_user, query, {}, default_roles_from_query, settings_from_query, grantees_from_query,allow_plaintext_password); + updateUserFromQueryImpl(*updated_user, query, {}, default_roles_from_query, settings_from_query, grantees_from_query, allow_plaintext_password); return updated_user; }; @@ -150,7 +139,7 @@ BlockIO InterpreterCreateUserQuery::execute() for (const auto & name : *query.names) { auto new_user = std::make_shared(); - updateUserFromQueryImpl(*new_user, query, name, default_roles_from_query, settings_from_query, RolesOrUsersSet::AllTag{},allow_plaintext_password); + updateUserFromQueryImpl(*new_user, query, name, default_roles_from_query, settings_from_query, RolesOrUsersSet::AllTag{}, allow_plaintext_password); new_users.emplace_back(std::move(new_user)); } @@ -178,9 +167,9 @@ BlockIO InterpreterCreateUserQuery::execute() } -void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreateUserQuery & query,const bool allow_plaintext_password) +void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreateUserQuery & query, const bool allow_plaintext_password) { - updateUserFromQueryImpl(user, query, {}, {}, {}, {},allow_plaintext_password); + updateUserFromQueryImpl(user, query, {}, {}, {}, {}, allow_plaintext_password); } } diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.h b/src/Interpreters/Access/InterpreterCreateUserQuery.h index 4798cfd795f..5594ead5dfd 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.h +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.h @@ -17,7 +17,7 @@ public: BlockIO execute() override; - static void updateUserFromQuery(User & user, const ASTCreateUserQuery & query,const bool allow_plaintext_password=1); + static void updateUserFromQuery(User & user, const ASTCreateUserQuery & query, const bool allow_plaintext_password=1); private: ASTPtr query_ptr; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index e8115c384bb..bba38b038e3 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -669,8 +669,19 @@ void Context::setUsersConfig(const ConfigurationPtr & config) shared->access_control->setUsersConfig(*shared->users_config); } -ConfigurationPtr Context::getUsersConfig() +void Context::setAllowPlaintextPasswordSetting(const bool allow_plaintext_password) { + auto lock = getLock(); + shared->access_control->setAllowPlaintextPasswordSetting(allow_plaintext_password); +} + +bool Context::getAllowPlaintextPasswordSetting() const +{ + return shared->access_control->getAllowPlaintextPasswordSetting(); +} + +ConfigurationPtr Context::getUsersConfig() +{ auto lock = getLock(); return shared->users_config; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 2a2783603a2..afef29a3eaa 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -383,6 +383,8 @@ public: * The accumulated quota values are not reset if the quota is not deleted. */ void setUsersConfig(const ConfigurationPtr & config); + void setAllowPlaintextPasswordSetting(const bool allow_plaintext_password_); + bool getAllowPlaintextPasswordSetting() const; ConfigurationPtr getUsersConfig(); /// Sets the current user assuming that he/she is already authenticated. diff --git a/src/Parsers/Access/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp index 13c0fcbb85d..18030a5ed80 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -36,7 +36,6 @@ namespace String auth_type_name = AuthenticationTypeInfo::get(auth_type).name; String by_keyword = "BY"; std::optional by_value; - if ( show_password || @@ -48,11 +47,6 @@ namespace { case AuthenticationType::PLAINTEXT_PASSWORD: { - - //bool allow_plaintext_password = auth_data.getPlaintextPasswordSetting(); - // std::cout<<"heena - You are at right place to throw exception."< - - - + + + ::1 127.0.0.1 diff --git a/tests/config/users.d/session_log_test.xml b/tests/config/users.d/session_log_test.xml index 4f5bb3e5bc0..d11c706e708 100644 --- a/tests/config/users.d/session_log_test.xml +++ b/tests/config/users.d/session_log_test.xml @@ -18,8 +18,8 @@ - - + + ::1 127.0.0.1 diff --git a/tests/queries/0_stateless/02207_allow_plaintext_password.config.xml b/tests/queries/0_stateless/02207_allow_plaintext_password.config.xml new file mode 100644 index 00000000000..a7493c7e7cb --- /dev/null +++ b/tests/queries/0_stateless/02207_allow_plaintext_password.config.xml @@ -0,0 +1,23 @@ + + + + trace + true + + + 9000 + 0 + . + 0 + + + + + users.xml + + + + ./ + + + diff --git a/tests/queries/0_stateless/02207_allow_plaintext_password.reference b/tests/queries/0_stateless/02207_allow_plaintext_password.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02207_allow_plaintext_password.sh b/tests/queries/0_stateless/02207_allow_plaintext_password.sh new file mode 100755 index 00000000000..5c6ca078f4a --- /dev/null +++ b/tests/queries/0_stateless/02207_allow_plaintext_password.sh @@ -0,0 +1,86 @@ +#!/usr/bin/env bash +# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-parallel, no-fasttest +# Tag no-tsan: requires jemalloc to track small allocations +# Tag no-asan: requires jemalloc to track small allocations +# Tag no-ubsan: requires jemalloc to track small allocations +# Tag no-msan: requires jemalloc to track small allocations + +# +# Regression for INSERT SELECT, that abnormally terminates the server +# in case of too small memory limits. +# +# NOTE: After #24483 had been merged the only place where the allocation may +# fail is the insert into PODArray in DB::OwnSplitChannel::log, but after +# #24069 those errors will be ignored, so to check new behaviour separate +# server is required. +# + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +cp "$CURDIR"/../../../programs/server/users.xml "$CURDIR"/users.xml + sed -i 's/<\/password>//g' "$CURDIR"/users.xml + sed -i 's//1<\/access_management>/g' "$CURDIR"/users.xml + +server_opts=( + "--config-file=$CURDIR/$(basename "${BASH_SOURCE[0]}" .sh).config.xml" + "--" + # to avoid multiple listen sockets (complexity for port discovering) + "--listen_host=127.1" + # we will discover the real port later. + "--tcp_port=0" + "--shutdown_wait_unfinished=0" +) + +CLICKHOUSE_WATCHDOG_ENABLE=0 $CLICKHOUSE_SERVER_BINARY "${server_opts[@]}" &> clickhouse-server.stderr & +server_pid=$! + +server_port= +i=0 retries=300 +# wait until server will start to listen (max 30 seconds) +while [[ -z $server_port ]] && [[ $i -lt $retries ]]; do + server_port=$(lsof -n -a -P -i tcp -s tcp:LISTEN -p $server_pid 2>/dev/null | awk -F'[ :]' '/LISTEN/ { print $(NF-1) }') + ((++i)) + sleep 0.1 + if ! kill -0 $server_pid >& /dev/null; then + echo "No server (pid $server_pid)" + break + fi +done +if [[ -z $server_port ]]; then + echo "Cannot wait for LISTEN socket" >&2 + exit 1 +fi + +# wait for the server to start accepting tcp connections (max 30 seconds) +i=0 retries=300 +while ! $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null -q 'select 1' 2>/dev/null && [[ $i -lt $retries ]]; do + sleep 0.1 + if ! kill -0 $server_pid >& /dev/null; then + echo "No server (pid $server_pid)" + break + fi +done + + +if ! $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null -q 'select 1'; then + echo "Cannot wait until server will start accepting connections on " >&2 + exit 1 +fi + +$CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" -q "CREATE USER u_02207 HOST IP '127.1' IDENTIFIED WITH plaintext_password BY 'qwerty' " " -- { serverError 43 } --" &> /dev/null ; + +# no sleep, since flushing to stderr should not be buffered. + grep 'User is not allowed to Create users with type PLAINTEXT_PASSWORD' clickhouse-server.stderr + + +# send TERM and save the error code to ensure that it is 0 (EXIT_SUCCESS) +kill $server_pid +wait $server_pid +return_code=$? + +rm -f clickhouse-server.stderr +rm -f "$CURDIR"/users.xml + +exit $return_code