Merge pull request #34738 from DevTeamBK/Issue-33953

New setting in Server Configuration to on/off AuthType Plaintext_password and No_password
This commit is contained in:
Vitaly Baranov 2022-03-14 17:09:46 +01:00 committed by GitHub
commit 4af61fb9d3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
18 changed files with 245 additions and 56 deletions

View File

@ -15,6 +15,7 @@
#include <base/scope_guard_safe.h>
#include <Interpreters/UserDefinedSQLObjectsLoader.h>
#include <Interpreters/Session.h>
#include <Access/AccessControl.h>
#include <Common/Exception.h>
#include <Common/Macros.h>
#include <Common/Config/ConfigProcessor.h>
@ -388,7 +389,9 @@ void LocalServer::setupUsers()
"</clickhouse>";
ConfigurationPtr users_config;
auto & access_control = global_context->getAccessControl();
access_control.setPlaintextPasswordSetting(config().getBool("allow_plaintext_password", true));
access_control.setNoPasswordSetting(config().getBool("allow_no_password", true));
if (config().has("users_config") || config().has("config-file") || fs::exists("config.xml"))
{
const auto users_config_path = config().getString("users_config", config().getString("config-file", "config.xml"));
@ -397,10 +400,7 @@ void LocalServer::setupUsers()
users_config = loaded_config.configuration;
}
else
{
users_config = getConfigurationFromXMLString(minimal_default_user_xml);
}
if (users_config)
global_context->setUsersConfig(users_config);
else
@ -802,7 +802,6 @@ void LocalServer::processOptions(const OptionsDescription &, const CommandLineOp
}
#pragma GCC diagnostic ignored "-Wunused-function"
#pragma GCC diagnostic ignored "-Wmissing-declarations"

View File

@ -1067,7 +1067,9 @@ if (ThreadFuzzer::instance().isEffective())
auto & access_control = global_context->getAccessControl();
if (config().has("custom_settings_prefixes"))
access_control.setCustomSettingsPrefixes(config().getString("custom_settings_prefixes"));
///set the allow_plaintext_and_no_password setting in context.
access_control.setPlaintextPasswordSetting(config().getBool("allow_plaintext_password", true));
access_control.setNoPasswordSetting(config().getBool("allow_no_password", true));
/// Initialize access storages.
try
{

View File

@ -243,7 +243,7 @@
openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096
Only file format with BEGIN DH PARAMETERS is supported.
-->
<!-- <dhParamsFile>/etc/clickhouse-server/dhparam.pem</dhParamsFile> -->
<!-- <dhParamsFile>/etc/clickhouse-server/dhparam.pem</dhParamsFile>-->
<verificationMode>none</verificationMode>
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
@ -367,6 +367,10 @@
<!-- Path to temporary data for processing hard queries. -->
<tmp_path>/var/lib/clickhouse/tmp/</tmp_path>
<!-- Disable AuthType Plaintext_password and No_password for ACL. -->
<!-- <allow_plaintext_password>0</allow_plaintext_password> -->
<!-- <allow_no_password>0</allow_no_password> -->`
<!-- Policy from the <storage_configuration> for the temporary files.
If not set <tmp_path> is used, otherwise <tmp_path> is ignored.

View File

@ -171,7 +171,9 @@ void AccessControl::addUsersConfigStorage(const Poco::Util::AbstractConfiguratio
void AccessControl::addUsersConfigStorage(const String & storage_name_, const Poco::Util::AbstractConfiguration & users_config_)
{
auto check_setting_name_function = [this](const std::string_view & setting_name) { checkSettingNameIsAllowed(setting_name); };
auto new_storage = std::make_shared<UsersConfigAccessStorage>(storage_name_, check_setting_name_function);
auto is_no_password_allowed_function = [this]() -> bool { return isNoPasswordAllowed(); };
auto is_plaintext_password_allowed_function = [this]() -> bool { return isPlaintextPasswordAllowed(); };
auto new_storage = std::make_shared<UsersConfigAccessStorage>(storage_name_, check_setting_name_function,is_no_password_allowed_function,is_plaintext_password_allowed_function);
new_storage->setConfig(users_config_);
addStorage(new_storage);
LOG_DEBUG(getLogger(), "Added {} access storage '{}', path: {}",
@ -205,7 +207,9 @@ void AccessControl::addUsersConfigStorage(
}
}
auto check_setting_name_function = [this](const std::string_view & setting_name) { checkSettingNameIsAllowed(setting_name); };
auto new_storage = std::make_shared<UsersConfigAccessStorage>(storage_name_, check_setting_name_function);
auto is_no_password_allowed_function = [this]() -> bool { return isNoPasswordAllowed(); };
auto is_plaintext_password_allowed_function = [this]() -> bool { return isPlaintextPasswordAllowed(); };
auto new_storage = std::make_shared<UsersConfigAccessStorage>(storage_name_, check_setting_name_function,is_no_password_allowed_function,is_plaintext_password_allowed_function);
new_storage->load(users_config_path_, include_from_path_, preprocessed_dir_, get_zookeeper_function_);
addStorage(new_storage);
LOG_DEBUG(getLogger(), "Added {} access storage '{}', path: {}", String(new_storage->getStorageType()), new_storage->getStorageName(), new_storage->getPath());
@ -407,7 +411,7 @@ UUID AccessControl::authenticate(const Credentials & credentials, const Poco::Ne
{
try
{
return MultipleAccessStorage::authenticate(credentials, address, *external_authenticators);
return MultipleAccessStorage::authenticate(credentials, address, *external_authenticators,allow_no_password, allow_plaintext_password);
}
catch (...)
{
@ -443,6 +447,15 @@ void AccessControl::setCustomSettingsPrefixes(const String & comma_separated_pre
setCustomSettingsPrefixes(prefixes);
}
void AccessControl::setPlaintextPasswordSetting(bool allow_plaintext_password_)
{
allow_plaintext_password = allow_plaintext_password_;
}
void AccessControl::setNoPasswordSetting(bool allow_no_password_)
{
allow_no_password = allow_no_password_;
}
bool AccessControl::isSettingNameAllowed(const std::string_view & setting_name) const
{
return custom_settings_prefixes->isSettingNameAllowed(setting_name);
@ -537,6 +550,15 @@ std::vector<QuotaUsage> AccessControl::getAllQuotasUsage() const
return quota_cache->getAllQuotasUsage();
}
bool AccessControl::isPlaintextPasswordAllowed() const
{
return allow_plaintext_password;
}
bool AccessControl::isNoPasswordAllowed() const
{
return allow_no_password;
}
std::shared_ptr<const EnabledSettings> AccessControl::getEnabledSettings(
const UUID & user_id,

View File

@ -4,6 +4,8 @@
#include <Common/SettingsChanges.h>
#include <Common/ZooKeeper/Common.h>
#include <boost/container/flat_set.hpp>
#include <Access/UsersConfigAccessStorage.h>
#include <memory>
@ -47,6 +49,8 @@ class AccessControl : public MultipleAccessStorage
public:
AccessControl();
~AccessControl() override;
std::atomic_bool allow_plaintext_password;
std::atomic_bool allow_no_password;
/// Parses access entities from a configuration loaded from users.xml.
/// This function add UsersConfigAccessStorage if it wasn't added before.
@ -72,7 +76,6 @@ public:
void reloadUsersConfigs();
void startPeriodicReloadingUsersConfigs();
void stopPeriodicReloadingUsersConfigs();
/// Loads access entities from the directory on the local disk.
/// Use that directory to keep created users/roles/etc.
void addDiskStorage(const String & directory_, bool readonly_ = false);
@ -113,6 +116,10 @@ public:
bool isSettingNameAllowed(const std::string_view & name) const;
void checkSettingNameIsAllowed(const std::string_view & name) const;
//sets allow_plaintext_password and allow_no_password setting
void setPlaintextPasswordSetting(const bool allow_plaintext_password_);
void setNoPasswordSetting(const bool allow_no_password_);
UUID authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address) const;
void setExternalAuthenticatorsConfig(const Poco::Util::AbstractConfiguration & config);
@ -146,6 +153,9 @@ public:
std::vector<QuotaUsage> getAllQuotasUsage() const;
bool isPlaintextPasswordAllowed() const;
bool isNoPasswordAllowed() const;
std::shared_ptr<const EnabledSettings> getEnabledSettings(
const UUID & user_id,
const SettingsProfileElements & settings_from_user,

View File

@ -23,6 +23,7 @@ namespace ErrorCodes
extern const int IP_ADDRESS_NOT_ALLOWED;
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
extern const int AUTHENTICATION_FAILED;
}
@ -440,9 +441,9 @@ void IAccessStorage::notify(const Notifications & notifications)
UUID IAccessStorage::authenticate(
const Credentials & credentials,
const Poco::Net::IPAddress & address,
const ExternalAuthenticators & external_authenticators) const
const ExternalAuthenticators & external_authenticators, bool allow_no_password, bool allow_plaintext_password) const
{
return *authenticateImpl(credentials, address, external_authenticators, /* throw_if_user_not_exists = */ true);
return *authenticateImpl(credentials, address, external_authenticators, /* throw_if_user_not_exists = */ true, allow_no_password, allow_plaintext_password);
}
@ -450,9 +451,9 @@ std::optional<UUID> IAccessStorage::authenticate(
const Credentials & credentials,
const Poco::Net::IPAddress & address,
const ExternalAuthenticators & external_authenticators,
bool throw_if_user_not_exists) const
bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const
{
return authenticateImpl(credentials, address, external_authenticators, throw_if_user_not_exists);
return authenticateImpl(credentials, address, external_authenticators, throw_if_user_not_exists, allow_no_password, allow_plaintext_password);
}
@ -460,7 +461,7 @@ std::optional<UUID> IAccessStorage::authenticateImpl(
const Credentials & credentials,
const Poco::Net::IPAddress & address,
const ExternalAuthenticators & external_authenticators,
bool throw_if_user_not_exists) const
bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const
{
if (auto id = find<User>(credentials.getUserName()))
{
@ -468,6 +469,8 @@ std::optional<UUID> IAccessStorage::authenticateImpl(
{
if (!isAddressAllowed(*user, address))
throwAddressNotAllowed(address);
if (isNoPasswordAllowed(*user, allow_no_password) || isPlaintextPasswordAllowed(*user, allow_plaintext_password))
throwPasswordTypeNotAllowed();
if (!areCredentialsValid(*user, credentials, external_authenticators))
throwInvalidCredentials();
@ -503,6 +506,15 @@ bool IAccessStorage::isAddressAllowed(const User & user, const Poco::Net::IPAddr
return user.allowed_client_hosts.contains(address);
}
bool IAccessStorage::isPlaintextPasswordAllowed(const User & user, bool allow_plaintext_password)
{
return !allow_plaintext_password && user.auth_data.getType() == AuthenticationType::PLAINTEXT_PASSWORD;
}
bool IAccessStorage::isNoPasswordAllowed(const User & user, bool allow_no_password)
{
return !allow_no_password && user.auth_data.getType() == AuthenticationType::NO_PASSWORD;
}
UUID IAccessStorage::generateRandomID()
{
@ -598,6 +610,12 @@ void IAccessStorage::throwAddressNotAllowed(const Poco::Net::IPAddress & address
throw Exception("Connections from " + address.toString() + " are not allowed", ErrorCodes::IP_ADDRESS_NOT_ALLOWED);
}
void IAccessStorage::throwPasswordTypeNotAllowed()
{
throw Exception(
"Authentication denied for users configured with AuthType PLAINTEXT_PASSWORD and NO_PASSWORD. Please check with Clickhouse admin to allow allow PLAINTEXT_PASSWORD and NO_PASSWORD through server configuration ",
ErrorCodes::AUTHENTICATION_FAILED);
}
void IAccessStorage::throwInvalidCredentials()
{
throw Exception("Invalid credentials", ErrorCodes::WRONG_PASSWORD);

View File

@ -148,8 +148,8 @@ public:
/// Finds a user, check the provided credentials and returns the ID of the user if they are valid.
/// Throws an exception if no such user or credentials are invalid.
UUID authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators) const;
std::optional<UUID> authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists) const;
UUID authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool allow_no_password=true, bool allow_plaintext_password=true) const;
std::optional<UUID> authenticate(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const;
protected:
virtual std::optional<UUID> findImpl(AccessEntityType type, const String & name) const = 0;
@ -161,10 +161,11 @@ protected:
virtual bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists);
virtual scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const = 0;
virtual scope_guard subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const = 0;
virtual std::optional<UUID> authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists) const;
virtual std::optional<UUID> authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const;
virtual bool areCredentialsValid(const User & user, const Credentials & credentials, const ExternalAuthenticators & external_authenticators) const;
virtual bool isAddressAllowed(const User & user, const Poco::Net::IPAddress & address) const;
static bool isPlaintextPasswordAllowed(const User & user, bool allow_plaintext_password) ;
static bool isNoPasswordAllowed(const User & user, bool allow_no_password);
static UUID generateRandomID();
Poco::Logger * getLogger() const;
static String formatEntityTypeWithName(AccessEntityType type, const String & name) { return AccessEntityTypeInfo::get(type).formatEntityNameWithType(name); }
@ -172,7 +173,7 @@ protected:
[[noreturn]] void throwNotFound(AccessEntityType type, const String & name) const;
[[noreturn]] static void throwBadCast(const UUID & id, AccessEntityType type, const String & name, AccessEntityType required_type);
[[noreturn]] void throwIDCollisionCannotInsert(
const UUID & id, AccessEntityType type, const String & name, AccessEntityType existing_type, const String & existing_name) const;
const UUID & id, AccessEntityType type, const String & name, AccessEntityType existing_type, const String & existing_name) const;
[[noreturn]] void throwNameCollisionCannotInsert(AccessEntityType type, const String & name) const;
[[noreturn]] void throwNameCollisionCannotRename(AccessEntityType type, const String & old_name, const String & new_name) const;
[[noreturn]] void throwReadonlyCannotInsert(AccessEntityType type, const String & name) const;
@ -180,7 +181,7 @@ protected:
[[noreturn]] void throwReadonlyCannotRemove(AccessEntityType type, const String & name) const;
[[noreturn]] static void throwAddressNotAllowed(const Poco::Net::IPAddress & address);
[[noreturn]] static void throwInvalidCredentials();
[[noreturn]] static void throwPasswordTypeNotAllowed();
using Notification = std::tuple<OnChangedHandler, UUID, AccessEntityPtr>;
using Notifications = std::vector<Notification>;
static void notify(const Notifications & notifications);

View File

@ -481,7 +481,7 @@ std::optional<UUID> LDAPAccessStorage::authenticateImpl(
const Credentials & credentials,
const Poco::Net::IPAddress & address,
const ExternalAuthenticators & external_authenticators,
bool throw_if_user_not_exists) const
bool throw_if_user_not_exists,bool allow_no_password __attribute__((unused)), bool allow_plaintext_password __attribute__((unused))) const
{
std::scoped_lock lock(mutex);
auto id = memory_storage.find<User>(credentials.getUserName());

View File

@ -52,7 +52,7 @@ private: // IAccessStorage implementations.
virtual std::optional<String> readNameImpl(const UUID & id, bool throw_if_not_exists) const override;
virtual scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override;
virtual scope_guard subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const override;
virtual std::optional<UUID> authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists) const override;
virtual std::optional<UUID> authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override;
private:
void setConfiguration(AccessControl * access_control_, const Poco::Util::AbstractConfiguration & config, const String & prefix);

View File

@ -449,14 +449,14 @@ void MultipleAccessStorage::updateSubscriptionsToNestedStorages(std::unique_lock
}
std::optional<UUID> MultipleAccessStorage::authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists) const
std::optional<UUID> MultipleAccessStorage::authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists,bool allow_no_password, bool allow_plaintext_password) const
{
auto storages = getStoragesInternal();
for (size_t i = 0; i != storages->size(); ++i)
{
const auto & storage = (*storages)[i];
bool is_last_storage = (i == storages->size() - 1);
auto id = storage->authenticate(credentials, address, external_authenticators, throw_if_user_not_exists && is_last_storage);
auto id = storage->authenticate(credentials, address, external_authenticators, (throw_if_user_not_exists && is_last_storage), allow_no_password, allow_plaintext_password);
if (id)
{
std::lock_guard lock{mutex};

View File

@ -50,7 +50,7 @@ protected:
bool updateImpl(const UUID & id, const UpdateFunc & update_func, bool throw_if_not_exists) override;
scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override;
scope_guard subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const override;
std::optional<UUID> authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists) const override;
std::optional<UUID> authenticateImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators, bool throw_if_user_not_exists, bool allow_no_password, bool allow_plaintext_password) const override;
private:
using Storages = std::vector<StoragePtr>;

View File

@ -28,11 +28,13 @@ 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
{
UUID generateID(AccessEntityType type, const String & name)
{
Poco::MD5Engine md5;
@ -52,9 +54,7 @@ namespace
{
auto user = std::make_shared<User>();
user->setName(user_name);
String user_config = "users." + user_name;
bool has_no_password = config.has(user_config + ".no_password");
bool has_password_plaintext = config.has(user_config + ".password");
bool has_password_sha256_hex = config.has(user_config + ".password_sha256_hex");
@ -66,6 +66,7 @@ namespace
bool has_certificates = config.has(certificates_config);
size_t num_password_fields = has_no_password + has_password_plaintext + has_password_sha256_hex + has_password_double_sha1_hex + has_ldap + has_kerberos + has_certificates;
if (num_password_fields > 1)
throw Exception("More than one field of 'password', 'password_sha256_hex', 'password_double_sha1_hex', 'no_password', 'ldap', 'kerberos', 'certificates' are used to specify authentication info for user " + user_name + ". Must be only one of them.",
ErrorCodes::BAD_ARGUMENTS);
@ -224,18 +225,23 @@ namespace
}
std::vector<AccessEntityPtr> parseUsers(const Poco::Util::AbstractConfiguration & config)
std::vector<AccessEntityPtr> parseUsers(const Poco::Util::AbstractConfiguration & config, Fn<bool()> auto && is_no_password_allowed_function, Fn<bool()> auto && is_plaintext_password_allowed_function)
{
Poco::Util::AbstractConfiguration::Keys user_names;
config.keys("users", user_names);
std::vector<AccessEntityPtr> users;
users.reserve(user_names.size());
bool allow_plaintext_password = is_plaintext_password_allowed_function();
bool allow_no_password = is_no_password_allowed_function();
for (const auto & user_name : user_names)
{
try
{
String user_config = "users." + user_name;
if ((config.has(user_config + ".password") && !allow_plaintext_password) || (config.has(user_config + ".no_password") && !allow_no_password))
throw Exception("Incorrect User configuration. User is not allowed to configure PLAINTEXT_PASSWORD or NO_PASSWORD. Please configure User with authtype SHA256_PASSWORD_HASH, SHA256_PASSWORD, DOUBLE_SHA1_PASSWORD OR enable setting allow_plaintext_and_no_password in server configuration to configure user with plaintext and no password Auth_Type"
" Though it is not recommended to use plaintext_password and No_password for user authentication.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
users.push_back(parseUser(config, user_name));
}
catch (Exception & e)
@ -508,14 +514,13 @@ namespace
}
}
UsersConfigAccessStorage::UsersConfigAccessStorage(const CheckSettingNameFunction & check_setting_name_function_)
: UsersConfigAccessStorage(STORAGE_TYPE, check_setting_name_function_)
UsersConfigAccessStorage::UsersConfigAccessStorage(const CheckSettingNameFunction & check_setting_name_function_, const IsNoPasswordFunction & is_no_password_allowed_function_, const IsPlaintextPasswordFunction & is_plaintext_password_allowed_function_)
: UsersConfigAccessStorage(STORAGE_TYPE, check_setting_name_function_, is_no_password_allowed_function_, is_plaintext_password_allowed_function_)
{
}
UsersConfigAccessStorage::UsersConfigAccessStorage(const String & storage_name_, const CheckSettingNameFunction & check_setting_name_function_)
: IAccessStorage(storage_name_), check_setting_name_function(check_setting_name_function_)
UsersConfigAccessStorage::UsersConfigAccessStorage(const String & storage_name_, const CheckSettingNameFunction & check_setting_name_function_, const IsNoPasswordFunction & is_no_password_allowed_function_, const IsPlaintextPasswordFunction & is_plaintext_password_allowed_function_)
: IAccessStorage(storage_name_), check_setting_name_function(check_setting_name_function_),is_no_password_allowed_function(is_no_password_allowed_function_), is_plaintext_password_allowed_function(is_plaintext_password_allowed_function_)
{
}
@ -534,7 +539,6 @@ String UsersConfigAccessStorage::getStorageParamsJSON() const
return oss.str();
}
String UsersConfigAccessStorage::getPath() const
{
std::lock_guard lock{load_mutex};
@ -546,7 +550,6 @@ bool UsersConfigAccessStorage::isPathEqual(const String & path_) const
return getPath() == path_;
}
void UsersConfigAccessStorage::setConfig(const Poco::Util::AbstractConfiguration & config)
{
std::lock_guard lock{load_mutex};
@ -560,7 +563,7 @@ void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfigu
try
{
std::vector<std::pair<UUID, AccessEntityPtr>> all_entities;
for (const auto & entity : parseUsers(config))
for (const auto & entity : parseUsers(config,is_no_password_allowed_function, is_plaintext_password_allowed_function))
all_entities.emplace_back(generateID(*entity), entity);
for (const auto & entity : parseQuotas(config))
all_entities.emplace_back(generateID(*entity), entity);
@ -595,6 +598,7 @@ void UsersConfigAccessStorage::load(
[&](Poco::AutoPtr<Poco::Util::AbstractConfiguration> new_config, bool /*initial_loading*/)
{
parseFromConfig(*new_config);
Settings::checkNoSettingNamesAtTopLevel(*new_config, users_config_path);
},
/* already_loaded = */ false);

View File

@ -18,11 +18,14 @@ class ConfigReloader;
class UsersConfigAccessStorage : public IAccessStorage
{
public:
static constexpr char STORAGE_TYPE[] = "users.xml";
using CheckSettingNameFunction = std::function<void(const std::string_view &)>;
using IsNoPasswordFunction = std::function<bool()>;
using IsPlaintextPasswordFunction = std::function<bool()>;
UsersConfigAccessStorage(const String & storage_name_ = STORAGE_TYPE, const CheckSettingNameFunction & check_setting_name_function_ = {});
UsersConfigAccessStorage(const CheckSettingNameFunction & check_setting_name_function_);
UsersConfigAccessStorage(const String & storage_name_ = STORAGE_TYPE, const CheckSettingNameFunction & check_setting_name_function_ = {}, const IsNoPasswordFunction & is_no_password_allowed_function_ ={}, const IsPlaintextPasswordFunction & is_plaintext_password_allowed_function_ = {});
UsersConfigAccessStorage(const CheckSettingNameFunction & check_setting_name_function_, const IsNoPasswordFunction & is_no_password_allowed_function_, const IsPlaintextPasswordFunction & is_plaintext_password_allowed_function_);
~UsersConfigAccessStorage() override;
const char * getStorageType() const override { return STORAGE_TYPE; }
@ -33,7 +36,6 @@ public:
bool isPathEqual(const String & path_) const;
void setConfig(const Poco::Util::AbstractConfiguration & config);
void load(const String & users_config_path,
const String & include_from_path = {},
const String & preprocessed_dir = {},
@ -48,7 +50,6 @@ public:
private:
void parseFromConfig(const Poco::Util::AbstractConfiguration & config);
std::optional<UUID> findImpl(AccessEntityType type, const String & name) const override;
std::vector<UUID> findAllImpl(AccessEntityType type) const override;
AccessEntityPtr readImpl(const UUID & id, bool throw_if_not_exists) const override;
@ -58,7 +59,8 @@ private:
MemoryAccessStorage memory_storage;
CheckSettingNameFunction check_setting_name_function;
IsNoPasswordFunction is_no_password_allowed_function;
IsPlaintextPasswordFunction is_plaintext_password_allowed_function;
String path;
std::unique_ptr<ConfigReloader> config_reloader;
mutable std::mutex load_mutex;

View File

@ -14,6 +14,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
namespace
{
void updateUserFromQueryImpl(
@ -22,7 +27,7 @@ namespace
const std::shared_ptr<ASTUserNameWithHost> & override_name,
const std::optional<RolesOrUsersSet> & override_default_roles,
const std::optional<SettingsProfileElements> & override_settings,
const std::optional<RolesOrUsersSet> & override_grantees)
const std::optional<RolesOrUsersSet> & override_grantees, bool allow_no_password, bool allow_plaintext_password)
{
if (override_name)
user.setName(override_name->toString());
@ -30,10 +35,15 @@ namespace
user.setName(query.new_name);
else if (query.names->size() == 1)
user.setName(query.names->front()->toString());
if (query.auth_data)
{
user.auth_data = *query.auth_data;
//User and query IDENTIFIED WITH AUTHTYPE PLAINTEXT and NO_PASSWORD should not be allowed if allow_plaintext_and_no_password is unset.
if ((query.auth_data->getType() == AuthenticationType::PLAINTEXT_PASSWORD && !allow_plaintext_password) || (query.auth_data->getType() == AuthenticationType::NO_PASSWORD && !allow_no_password))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "User is not allowed to ALTER/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_and_no_password in server configuration to configure user with " + toString(query.auth_data->getType()) +" Auth_type."
+ "It is not recommended to use " + toString(query.auth_data->getType()) + ".");
}
if (override_name && !override_name->host_pattern.empty())
{
user.allowed_client_hosts = AllowedClientHosts{};
@ -75,13 +85,14 @@ namespace
}
}
BlockIO InterpreterCreateUserQuery::execute()
{
const auto & query = query_ptr->as<const ASTCreateUserQuery &>();
auto & access_control = getContext()->getAccessControl();
auto access = getContext()->getAccess();
access->checkAccess(query.alter ? AccessType::ALTER_USER : AccessType::CREATE_USER);
bool allow_plaintext_password = access_control.isPlaintextPasswordAllowed();
bool allow_no_password = access_control.isNoPasswordAllowed();
std::optional<RolesOrUsersSet> default_roles_from_query;
if (query.default_roles)
@ -93,10 +104,8 @@ BlockIO InterpreterCreateUserQuery::execute()
access->checkAdminOption(role);
}
}
if (!query.cluster.empty())
return executeDDLQueryOnCluster(query_ptr, getContext());
std::optional<SettingsProfileElements> settings_from_query;
if (query.settings)
settings_from_query = SettingsProfileElements{*query.settings, access_control};
@ -110,7 +119,7 @@ BlockIO InterpreterCreateUserQuery::execute()
auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr
{
auto updated_user = typeid_cast<std::shared_ptr<User>>(entity->clone());
updateUserFromQueryImpl(*updated_user, query, {}, default_roles_from_query, settings_from_query, grantees_from_query);
updateUserFromQueryImpl(*updated_user, query, {}, default_roles_from_query, settings_from_query, grantees_from_query, allow_no_password, allow_plaintext_password);
return updated_user;
};
@ -129,7 +138,7 @@ BlockIO InterpreterCreateUserQuery::execute()
for (const auto & name : *query.names)
{
auto new_user = std::make_shared<User>();
updateUserFromQueryImpl(*new_user, query, name, default_roles_from_query, settings_from_query, RolesOrUsersSet::AllTag{});
updateUserFromQueryImpl(*new_user, query, name, default_roles_from_query, settings_from_query, RolesOrUsersSet::AllTag{}, allow_no_password, allow_plaintext_password);
new_users.emplace_back(std::move(new_user));
}
@ -157,9 +166,9 @@ BlockIO InterpreterCreateUserQuery::execute()
}
void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreateUserQuery & query)
void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreateUserQuery & query, bool allow_no_password, bool allow_plaintext_password)
{
updateUserFromQueryImpl(user, query, {}, {}, {}, {});
updateUserFromQueryImpl(user, query, {}, {}, {}, {}, allow_no_password, allow_plaintext_password);
}
}

View File

@ -17,7 +17,7 @@ public:
BlockIO execute() override;
static void updateUserFromQuery(User & user, const ASTCreateUserQuery & query);
static void updateUserFromQuery(User & user, const ASTCreateUserQuery & query, bool allow_no_password=true, bool allow_plaintext_password=true);
private:
ASTPtr query_ptr;

View File

@ -0,0 +1,24 @@
<?xml version="1.0"?>
<clickhouse>
<logger>
<level>trace</level>
<console>true</console>
</logger>
<tcp_port>9000</tcp_port>
<allow_plaintext_password>0</allow_plaintext_password>
<allow_no_password>0</allow_no_password>
<path>.</path>
<mark_cache_size>0</mark_cache_size>
<!-- Sources to read users, roles, access rights, profiles of settings, quotas. -->
<user_directories>
<users_xml>
<!-- Path to configuration file with predefined users. -->
<path>users.xml</path>
</users_xml>
<local_directory>
<!-- Path to folder where users created by SQL commands are stored. -->
<path>./</path>
</local_directory>
</user_directories>
</clickhouse>

View File

@ -0,0 +1,94 @@
#!/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
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
cp /etc/clickhouse-server/users.xml "$CURDIR"/users.xml
sed -i 's/<password><\/password>/<password_sha256_hex>c64c5e4e53ea1a9f1427d2713b3a22bbebe8940bc807adaf654744b1568c70ab<\/password_sha256_hex>/g' "$CURDIR"/users.xml
sed -i 's/<!-- <access_management>1<\/access_management> -->/<access_management>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 -u default --password='1w2swhb1' --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 -u default --password='1w2swhb1' --host 127.1 --port "$server_port" --format Null -q 'select 1'; then
echo "Cannot wait until server will start accepting connections on <tcp_port>" >&2
exit 1
fi
$CLICKHOUSE_CLIENT_BINARY -u default --password='1w2swhb1' --host 127.1 --port "$server_port" -q " DROP USER IF EXISTS u_02207, u1_02207";
$CLICKHOUSE_CLIENT_BINARY -u default --password='1w2swhb1' --host 127.1 --port "$server_port" -q "CREATE USER u_02207 IDENTIFIED WITH double_sha1_hash BY '8DCDD69CE7D121DE8013062AEAEB2A148910D50E'
"
$CLICKHOUSE_CLIENT_BINARY -u default --password='1w2swhb1' --host 127.1 --port "$server_port" -q " CREATE USER u1_02207 IDENTIFIED BY 'qwe123'";
$CLICKHOUSE_CLIENT_BINARY -u default --password='1w2swhb1' --host 127.1 --port "$server_port" -q "CREATE USER u2_02207 HOST IP '127.1' IDENTIFIED WITH plaintext_password BY 'qwerty' " " -- { serverError 516 } --" &> /dev/null ;
$CLICKHOUSE_CLIENT_BINARY -u default --password='1w2swhb1' --host 127.1 --port "$server_port" -q "CREATE USER u3_02207 HOST IP '127.1' IDENTIFIED WITH no_password " " -- { serverError 516 } --" &> /dev/null ;
$CLICKHOUSE_CLIENT_BINARY -u default --password='1w2swhb1' --host 127.1 --port "$server_port" -q "CREATE USER u4_02207 HOST IP '127.1' NOT IDENTIFIED " " -- { serverError 516 } --" &> /dev/null ;
$CLICKHOUSE_CLIENT_BINARY -u default --password='1w2swhb1' --host 127.1 --port "$server_port" -q "CREATE USER IF NOT EXISTS u5_02207 " " -- { serverError 516 } --" &> /dev/null ;
$CLICKHOUSE_CLIENT_BINARY -u default --password='1w2swhb1' --host 127.1 --port "$server_port" -q " DROP USER u_02207, u1_02207";
# no sleep, since flushing to stderr should not be buffered.
grep 'User is not allowed to Create users' 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