Fix: System Reload Config Failure

This commit is contained in:
HeenaBansal2009 2022-02-28 10:51:49 -08:00
parent 1b263f0c15
commit aa8494a808
14 changed files with 84 additions and 69 deletions

View File

@ -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"))
{

View File

@ -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
{

View File

@ -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.

View File

@ -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();

View File

@ -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;
};
}

View File

@ -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);
},

View File

@ -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;
};
}

View File

@ -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);
}
}

View File

@ -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;

View File

@ -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;
}

View File

@ -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.

View File

@ -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. -->

View File

@ -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)