mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Fix: System Reload Config Failure
This commit is contained in:
parent
1b263f0c15
commit
aa8494a808
@ -390,7 +390,7 @@ void LocalServer::setupUsers()
|
||||
ConfigurationPtr users_config;
|
||||
|
||||
//set the allow_plaintext_password setting in global context.
|
||||
global_context->setAllowPlaintextPasswordSetting(config().getBool("allow_plaintext_password",1));
|
||||
global_context->setAuthTypeSetting(config().getBool("allow_plaintext_and_no_password",true));
|
||||
|
||||
if (config().has("users_config") || config().has("config-file") || fs::exists("config.xml"))
|
||||
{
|
||||
|
@ -1069,9 +1069,8 @@ 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_password setting in context.
|
||||
access_control.setAllowPlaintextPasswordSetting(config().getBool("allow_plaintext_password", 1));
|
||||
|
||||
///set the allow_plaintext_and_no_password setting in context.
|
||||
global_context->setAuthTypeSetting(config().getBool("allow_plaintext_and_no_password",true));
|
||||
/// Initialize access storages.
|
||||
try
|
||||
{
|
||||
|
@ -368,8 +368,8 @@
|
||||
<!-- Path to temporary data for processing hard queries. -->
|
||||
<tmp_path>/var/lib/clickhouse/tmp/</tmp_path>
|
||||
|
||||
<!-- Disable AuthType Plaintext_password for ACL. -->
|
||||
<!-- <allow_plaintext_password>0<allow_plaintext_password> -->
|
||||
<!-- Disable AuthType Plaintext_password and No_password AUTH_TYPE for ACL. -->
|
||||
<!-- <allow_plaintext_and_no_password>0</allow_plaintext_and_no_password> -->
|
||||
|
||||
<!-- Policy from the <storage_configuration> for the temporary files.
|
||||
If not set <tmp_path> is used, otherwise <tmp_path> is ignored.
|
||||
|
@ -156,7 +156,7 @@ void AccessControl::setUsersConfig(const Poco::Util::AbstractConfiguration & use
|
||||
{
|
||||
if (auto users_config_storage = typeid_cast<std::shared_ptr<UsersConfigAccessStorage>>(storage))
|
||||
{
|
||||
users_config_storage->setConfig(users_config_);
|
||||
users_config_storage->setConfig(users_config_);//,allow_plaintext_and_no_password);
|
||||
return;
|
||||
}
|
||||
}
|
||||
@ -172,7 +172,8 @@ 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<UsersConfigAccessStorage>(storage_name_, check_setting_name_function);
|
||||
new_storage->setConfig(users_config_, allow_plaintext_password);
|
||||
// auto new_storage = std::make_shared<UsersConfigAccessStorage>(storage_name_, check_setting_name_function,allow_plaintext_and_no_password);
|
||||
new_storage->setConfig(users_config_);//, allow_plaintext_and_no_password);
|
||||
addStorage(new_storage);
|
||||
LOG_DEBUG(getLogger(), "Added {} access storage '{}', path: {}",
|
||||
String(new_storage->getStorageType()), new_storage->getStorageName(), new_storage->getPath());
|
||||
@ -206,18 +207,24 @@ 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);
|
||||
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_);
|
||||
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_)
|
||||
void AccessControl::setAuthTypeSetting(const bool allow_plaintext_and_no_password)
|
||||
{
|
||||
allow_plaintext_password = allow_plaintext_password_;
|
||||
UsersConfigAccessStorage::setAuthTypeSetting(allow_plaintext_and_no_password);
|
||||
}
|
||||
|
||||
bool AccessControl::getAuthTypeSetting() const
|
||||
{
|
||||
return UsersConfigAccessStorage::ALLOW_PLAINTEXT_AND_NO_PASSWORD;
|
||||
}
|
||||
|
||||
void AccessControl::reloadUsersConfigs()
|
||||
{
|
||||
auto storages = getStoragesPtr();
|
||||
for (const auto & storage : *storages)
|
||||
for (const auto & storage : *storages)
|
||||
{
|
||||
if (auto users_config_storage = typeid_cast<std::shared_ptr<UsersConfigAccessStorage>>(storage))
|
||||
users_config_storage->reload();
|
||||
|
@ -47,7 +47,7 @@ class AccessControl : public MultipleAccessStorage
|
||||
public:
|
||||
AccessControl();
|
||||
~AccessControl() override;
|
||||
|
||||
//bool allow_plaintext_and_no_password;
|
||||
/// Parses access entities from a configuration loaded from users.xml.
|
||||
/// This function add UsersConfigAccessStorage if it wasn't added before.
|
||||
void setUsersConfig(const Poco::Util::AbstractConfiguration & users_config_);
|
||||
@ -144,8 +144,8 @@ 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; }
|
||||
void setAuthTypeSetting(const bool allow_plaintext_and_no_password);
|
||||
bool getAuthTypeSetting() const ;
|
||||
|
||||
std::vector<QuotaUsage> getAllQuotasUsage() const;
|
||||
|
||||
@ -170,7 +170,6 @@ private:
|
||||
std::unique_ptr<SettingsProfilesCache> settings_profiles_cache;
|
||||
std::unique_ptr<ExternalAuthenticators> external_authenticators;
|
||||
std::unique_ptr<CustomSettingsPrefixes> custom_settings_prefixes;
|
||||
bool allow_plaintext_password;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -34,6 +34,7 @@ namespace ErrorCodes
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
UUID generateID(AccessEntityType type, const String & name)
|
||||
{
|
||||
Poco::MD5Engine md5;
|
||||
@ -49,20 +50,17 @@ namespace
|
||||
UUID generateID(const IAccessEntity & entity) { return generateID(entity.getType(), entity.getName()); }
|
||||
|
||||
|
||||
UserPtr parseUser(const Poco::Util::AbstractConfiguration & config, const String & user_name , const bool allow_plaintext_password_)
|
||||
UserPtr parseUser(const Poco::Util::AbstractConfiguration & config, const String & user_name)
|
||||
{
|
||||
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");
|
||||
bool has_password_double_sha1_hex = config.has(user_config + ".password_double_sha1_hex");
|
||||
bool has_ldap = config.has(user_config + ".ldap");
|
||||
bool has_kerberos = config.has(user_config + ".kerberos");
|
||||
|
||||
size_t num_password_fields = has_no_password + has_password_plaintext + has_password_sha256_hex + has_password_double_sha1_hex + has_ldap + has_kerberos;
|
||||
if (num_password_fields > 1)
|
||||
throw Exception("More than one field of 'password', 'password_sha256_hex', 'password_double_sha1_hex', 'no_password', 'ldap', 'kerberos' are used to specify password for user " + user_name + ". Must be only one of them.",
|
||||
@ -71,12 +69,11 @@ 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 allow_plaintext_password = allow_plaintext_password_;
|
||||
if ((has_password_plaintext || has_no_password) && !UsersConfigAccessStorage::ALLOW_PLAINTEXT_AND_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);
|
||||
if (has_password_plaintext)
|
||||
{
|
||||
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"));
|
||||
}
|
||||
@ -206,7 +203,7 @@ namespace
|
||||
}
|
||||
|
||||
|
||||
std::vector<AccessEntityPtr> parseUsers(const Poco::Util::AbstractConfiguration & config, const bool allow_plaintext_password)
|
||||
std::vector<AccessEntityPtr> parseUsers(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
Poco::Util::AbstractConfiguration::Keys user_names;
|
||||
config.keys("users", user_names);
|
||||
@ -218,7 +215,7 @@ namespace
|
||||
{
|
||||
try
|
||||
{
|
||||
users.push_back(parseUser(config, user_name, allow_plaintext_password));
|
||||
users.push_back(parseUser(config, user_name));
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
@ -490,7 +487,7 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
bool UsersConfigAccessStorage::ALLOW_PLAINTEXT_AND_NO_PASSWORD=true;
|
||||
UsersConfigAccessStorage::UsersConfigAccessStorage(const CheckSettingNameFunction & check_setting_name_function_)
|
||||
: UsersConfigAccessStorage(STORAGE_TYPE, check_setting_name_function_)
|
||||
{
|
||||
@ -516,7 +513,6 @@ String UsersConfigAccessStorage::getStorageParamsJSON() const
|
||||
return oss.str();
|
||||
}
|
||||
|
||||
|
||||
String UsersConfigAccessStorage::getPath() const
|
||||
{
|
||||
std::lock_guard lock{load_mutex};
|
||||
@ -528,21 +524,20 @@ bool UsersConfigAccessStorage::isPathEqual(const String & path_) const
|
||||
return getPath() == path_;
|
||||
}
|
||||
|
||||
|
||||
void UsersConfigAccessStorage::setConfig(const Poco::Util::AbstractConfiguration & config, const bool allow_plaintext_password)
|
||||
void UsersConfigAccessStorage::setConfig(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
std::lock_guard lock{load_mutex};
|
||||
path.clear();
|
||||
config_reloader.reset();
|
||||
parseFromConfig(config, allow_plaintext_password);
|
||||
parseFromConfig(config);
|
||||
}
|
||||
|
||||
void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfiguration & config, const bool allow_plaintext_password)
|
||||
void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
try
|
||||
{
|
||||
std::vector<std::pair<UUID, AccessEntityPtr>> all_entities;
|
||||
for (const auto & entity : parseUsers(config, allow_plaintext_password))
|
||||
for (const auto & entity : parseUsers(config))
|
||||
all_entities.emplace_back(generateID(*entity), entity);
|
||||
for (const auto & entity : parseQuotas(config))
|
||||
all_entities.emplace_back(generateID(*entity), entity);
|
||||
@ -563,7 +558,7 @@ 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_)
|
||||
const zkutil::GetZooKeeper & get_zookeeper_function)
|
||||
{
|
||||
std::lock_guard lock{load_mutex};
|
||||
path = std::filesystem::path{users_config_path}.lexically_normal();
|
||||
@ -576,7 +571,7 @@ void UsersConfigAccessStorage::load(
|
||||
std::make_shared<Poco::Event>(),
|
||||
[&](Poco::AutoPtr<Poco::Util::AbstractConfiguration> new_config, bool /*initial_loading*/)
|
||||
{
|
||||
parseFromConfig(*new_config, allow_plaintext_password_);
|
||||
parseFromConfig(*new_config);
|
||||
|
||||
Settings::checkNoSettingNamesAtTopLevel(*new_config, users_config_path);
|
||||
},
|
||||
|
@ -18,6 +18,7 @@ class ConfigReloader;
|
||||
class UsersConfigAccessStorage : public IAccessStorage
|
||||
{
|
||||
public:
|
||||
static bool ALLOW_PLAINTEXT_AND_NO_PASSWORD;
|
||||
static constexpr char STORAGE_TYPE[] = "users.xml";
|
||||
using CheckSettingNameFunction = std::function<void(const std::string_view &)>;
|
||||
|
||||
@ -32,12 +33,13 @@ public:
|
||||
String getPath() const;
|
||||
bool isPathEqual(const String & path_) const;
|
||||
|
||||
void setConfig(const Poco::Util::AbstractConfiguration & config, const bool allow_plaintext_password=1);
|
||||
void setConfig(const Poco::Util::AbstractConfiguration & config);
|
||||
static void setAuthTypeSetting(const bool allow_plaintext_and_no_password_) { UsersConfigAccessStorage::ALLOW_PLAINTEXT_AND_NO_PASSWORD=allow_plaintext_and_no_password_;}
|
||||
|
||||
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=1);
|
||||
const zkutil::GetZooKeeper & get_zookeeper_function = {});
|
||||
void reload();
|
||||
void startPeriodicReloading();
|
||||
void stopPeriodicReloading();
|
||||
@ -47,8 +49,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);
|
||||
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;
|
||||
@ -61,5 +62,6 @@ private:
|
||||
String path;
|
||||
std::unique_ptr<ConfigReloader> config_reloader;
|
||||
mutable std::mutex load_mutex;
|
||||
// bool allow_plaintext_and_no_password;
|
||||
};
|
||||
}
|
||||
|
@ -27,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 bool allow_plaintext_password)
|
||||
const std::optional<RolesOrUsersSet> & override_grantees, const bool allow_plaintext_and_no_password)
|
||||
{
|
||||
if (override_name)
|
||||
user.setName(override_name->toString());
|
||||
@ -35,15 +35,14 @@ namespace
|
||||
user.setName(query.new_name);
|
||||
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 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;
|
||||
}
|
||||
|
||||
//User and query IDENTIFIED WITH AUTHTYPE PLAINTEXT and NO_PASSWORD should not be allowed if llow_plaintext_and_no_password is unset.
|
||||
if ((user.auth_data.getType() == AuthenticationType::NO_PASSWORD || query.auth_data->getType() == AuthenticationType::PLAINTEXT_PASSWORD || query.auth_data->getType() == AuthenticationType::NO_PASSWORD) && !allow_plaintext_and_no_password)
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "InterPreterQuery: User is not allowed to ALTER/CREATE USERS with type "+ toString(user.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 plaintext_password and no_password Auth_type."
|
||||
+ "It is not recommended to use " + toString(user.auth_data.getType()) + ".");
|
||||
if (override_name && !override_name->host_pattern.empty())
|
||||
{
|
||||
user.allowed_client_hosts = AllowedClientHosts{};
|
||||
@ -91,7 +90,7 @@ BlockIO InterpreterCreateUserQuery::execute()
|
||||
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.getAllowPlaintextPasswordSetting();
|
||||
bool allow_plaintext_and_no_password = access_control.getAuthTypeSetting();
|
||||
|
||||
std::optional<RolesOrUsersSet> default_roles_from_query;
|
||||
if (query.default_roles)
|
||||
@ -103,10 +102,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};
|
||||
@ -120,7 +117,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, allow_plaintext_password);
|
||||
updateUserFromQueryImpl(*updated_user, query, {}, default_roles_from_query, settings_from_query, grantees_from_query, allow_plaintext_and_no_password);
|
||||
return updated_user;
|
||||
};
|
||||
|
||||
@ -139,7 +136,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{}, allow_plaintext_password);
|
||||
updateUserFromQueryImpl(*new_user, query, name, default_roles_from_query, settings_from_query, RolesOrUsersSet::AllTag{}, allow_plaintext_and_no_password);
|
||||
new_users.emplace_back(std::move(new_user));
|
||||
}
|
||||
|
||||
@ -167,9 +164,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_and_no_password)
|
||||
{
|
||||
updateUserFromQueryImpl(user, query, {}, {}, {}, {}, allow_plaintext_password);
|
||||
updateUserFromQueryImpl(user, query, {}, {}, {}, {}, allow_plaintext_and_no_password);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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_and_no_password=true);
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
|
@ -669,19 +669,19 @@ void Context::setUsersConfig(const ConfigurationPtr & config)
|
||||
shared->access_control->setUsersConfig(*shared->users_config);
|
||||
}
|
||||
|
||||
void Context::setAllowPlaintextPasswordSetting(const bool allow_plaintext_password)
|
||||
void Context::setAuthTypeSetting(const bool allow_plaintext_and_no_password)
|
||||
{
|
||||
auto lock = getLock();
|
||||
shared->access_control->setAllowPlaintextPasswordSetting(allow_plaintext_password);
|
||||
shared->access_control->setAuthTypeSetting(allow_plaintext_and_no_password);
|
||||
}
|
||||
|
||||
bool Context::getAllowPlaintextPasswordSetting() const
|
||||
bool Context::getAuthTypeSetting() const
|
||||
{
|
||||
return shared->access_control->getAllowPlaintextPasswordSetting();
|
||||
return shared->access_control->getAuthTypeSetting();
|
||||
}
|
||||
|
||||
ConfigurationPtr Context::getUsersConfig()
|
||||
{
|
||||
{
|
||||
auto lock = getLock();
|
||||
return shared->users_config;
|
||||
}
|
||||
|
@ -383,8 +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;
|
||||
void setAuthTypeSetting(const bool allow_plaintext_and_no_password_);
|
||||
bool getAuthTypeSetting() const;
|
||||
ConfigurationPtr getUsersConfig();
|
||||
|
||||
/// Sets the current user assuming that he/she is already authenticated.
|
||||
|
@ -6,7 +6,7 @@
|
||||
</logger>
|
||||
|
||||
<tcp_port>9000</tcp_port>
|
||||
<allow_plaintext_password>0</allow_plaintext_password>
|
||||
<allow_plaintext_and_no_password>0</allow_plaintext_and_no_password>
|
||||
<path>.</path>
|
||||
<mark_cache_size>0</mark_cache_size>
|
||||
<!-- Sources to read users, roles, access rights, profiles of settings, quotas. -->
|
@ -19,8 +19,8 @@ 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><\/password>/<no_password\/>/g' "$CURDIR"/users.xml
|
||||
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=(
|
||||
@ -55,7 +55,7 @@ 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
|
||||
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)"
|
||||
@ -64,15 +64,31 @@ while ! $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Nu
|
||||
done
|
||||
|
||||
|
||||
if ! $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null -q 'select 1'; then
|
||||
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 --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 ;
|
||||
$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 43 } --" &> /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 43 } --" &> /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 43 } --" &> /dev/null ;
|
||||
|
||||
$CLICKHOUSE_CLIENT_BINARY -u default --password='1w2swhb1' --host 127.1 --port "$server_port" -q "CREATE USER IF NOT EXISTS u5_02207 " " -- { serverError 43 } --" &> /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 with type PLAINTEXT_PASSWORD' clickhouse-server.stderr
|
||||
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)
|
Loading…
Reference in New Issue
Block a user