Merge pull request #43719 from evillique/password_complexity_rules

Add password complexity rules
This commit is contained in:
Nikolay Degterinsky 2022-12-13 17:56:45 +01:00 committed by GitHub
commit 443bc3e656
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 302 additions and 4 deletions

View File

@ -16,6 +16,8 @@
#include <base/find_symbols.h> #include <base/find_symbols.h>
#include <Access/AccessControl.h>
#include "config_version.h" #include "config_version.h"
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/formatReadable.h> #include <Common/formatReadable.h>
@ -258,6 +260,10 @@ try
if (is_interactive && !config().has("no-warnings")) if (is_interactive && !config().has("no-warnings"))
showWarnings(); showWarnings();
/// Set user password complexity rules
auto & access_control = global_context->getAccessControl();
access_control.setPasswordComplexityRules(connection->getPasswordComplexityRules());
if (is_interactive && !delayed_interactive) if (is_interactive && !delayed_interactive)
{ {
runInteractive(); runInteractive();

View File

@ -466,6 +466,30 @@
<allow_no_password>1</allow_no_password> <allow_no_password>1</allow_no_password>
<allow_implicit_no_password>1</allow_implicit_no_password> <allow_implicit_no_password>1</allow_implicit_no_password>
<!-- Complexity requirements for user passwords. -->
<!-- <password_complexity>
<rule>
<pattern>.{12}</pattern>
<message>be at least 12 characters long</message>
</rule>
<rule>
<pattern>\p{N}</pattern>
<message>contain at least 1 numeric character</message>
</rule>
<rule>
<pattern>\p{Ll}</pattern>
<message>contain at least 1 lowercase character</message>
</rule>
<rule>
<pattern>\p{Lu}</pattern>
<message>contain at least 1 uppercase character</message>
</rule>
<rule>
<pattern>[^\p{L}\p{N}]</pattern>
<message>contain at least 1 special character</message>
</rule>
</password_complexity> -->
<!-- Policy from the <storage_configuration> for the temporary files. <!-- Policy from the <storage_configuration> for the temporary files.
If not set <tmp_path> is used, otherwise <tmp_path> is ignored. If not set <tmp_path> is used, otherwise <tmp_path> is ignored.

View File

@ -27,6 +27,7 @@
#include <boost/algorithm/string/join.hpp> #include <boost/algorithm/string/join.hpp>
#include <boost/algorithm/string/split.hpp> #include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/trim.hpp> #include <boost/algorithm/string/trim.hpp>
#include <re2/re2.h>
#include <filesystem> #include <filesystem>
#include <mutex> #include <mutex>
@ -38,6 +39,8 @@ namespace ErrorCodes
extern const int UNKNOWN_ELEMENT_IN_CONFIG; extern const int UNKNOWN_ELEMENT_IN_CONFIG;
extern const int UNKNOWN_SETTING; extern const int UNKNOWN_SETTING;
extern const int AUTHENTICATION_FAILED; extern const int AUTHENTICATION_FAILED;
extern const int CANNOT_COMPILE_REGEXP;
extern const int BAD_ARGUMENTS;
} }
namespace namespace
@ -140,6 +143,109 @@ private:
}; };
class AccessControl::PasswordComplexityRules
{
public:
void setPasswordComplexityRulesFromConfig(const Poco::Util::AbstractConfiguration & config_)
{
std::lock_guard lock{mutex};
rules.clear();
if (config_.has("password_complexity"))
{
Poco::Util::AbstractConfiguration::Keys password_complexity;
config_.keys("password_complexity", password_complexity);
for (const auto & key : password_complexity)
{
if (key == "rule" || key.starts_with("rule["))
{
String pattern(config_.getString("password_complexity." + key + ".pattern"));
String message(config_.getString("password_complexity." + key + ".message"));
auto matcher = std::make_unique<RE2>(pattern, RE2::Quiet);
if (!matcher->ok())
throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP,
"Password complexity pattern {} cannot be compiled: {}",
pattern, matcher->error());
rules.push_back({std::move(matcher), std::move(pattern), std::move(message)});
}
}
}
}
void setPasswordComplexityRules(const std::vector<std::pair<String, String>> & rules_)
{
Rules new_rules;
for (const auto & [original_pattern, exception_message] : rules_)
{
auto matcher = std::make_unique<RE2>(original_pattern, RE2::Quiet);
if (!matcher->ok())
throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP,
"Password complexity pattern {} cannot be compiled: {}",
original_pattern, matcher->error());
new_rules.push_back({std::move(matcher), original_pattern, exception_message});
}
std::lock_guard lock{mutex};
rules = std::move(new_rules);
}
void checkPasswordComplexityRules(const String & password_) const
{
String exception_text;
bool failed = false;
std::lock_guard lock{mutex};
for (const auto & rule : rules)
{
if (!RE2::PartialMatch(password_, *rule.matcher))
{
failed = true;
if (!exception_text.empty())
exception_text += ", ";
exception_text += rule.exception_message;
}
}
if (failed)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid password. The password should: {}", exception_text);
}
std::vector<std::pair<String, String>> getPasswordComplexityRules()
{
std::vector<std::pair<String, String>> result;
std::lock_guard lock{mutex};
result.reserve(rules.size());
for (const auto & rule : rules)
result.push_back({rule.original_pattern, rule.exception_message});
return result;
}
private:
struct Rule
{
std::unique_ptr<RE2> matcher;
String original_pattern;
String exception_message;
};
using Rules = std::vector<Rule>;
Rules rules TSA_GUARDED_BY(mutex);
mutable std::mutex mutex;
};
AccessControl::AccessControl() AccessControl::AccessControl()
: MultipleAccessStorage("user directories"), : MultipleAccessStorage("user directories"),
context_access_cache(std::make_unique<ContextAccessCache>(*this)), context_access_cache(std::make_unique<ContextAccessCache>(*this)),
@ -149,7 +255,8 @@ AccessControl::AccessControl()
settings_profiles_cache(std::make_unique<SettingsProfilesCache>(*this)), settings_profiles_cache(std::make_unique<SettingsProfilesCache>(*this)),
external_authenticators(std::make_unique<ExternalAuthenticators>()), external_authenticators(std::make_unique<ExternalAuthenticators>()),
custom_settings_prefixes(std::make_unique<CustomSettingsPrefixes>()), custom_settings_prefixes(std::make_unique<CustomSettingsPrefixes>()),
changes_notifier(std::make_unique<AccessChangesNotifier>()) changes_notifier(std::make_unique<AccessChangesNotifier>()),
password_rules(std::make_unique<PasswordComplexityRules>())
{ {
} }
@ -166,6 +273,7 @@ void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration
setImplicitNoPasswordAllowed(config_.getBool("allow_implicit_no_password", true)); setImplicitNoPasswordAllowed(config_.getBool("allow_implicit_no_password", true));
setNoPasswordAllowed(config_.getBool("allow_no_password", true)); setNoPasswordAllowed(config_.getBool("allow_no_password", true));
setPlaintextPasswordAllowed(config_.getBool("allow_plaintext_password", true)); setPlaintextPasswordAllowed(config_.getBool("allow_plaintext_password", true));
setPasswordComplexityRulesFromConfig(config_);
/// Optional improvements in access control system. /// Optional improvements in access control system.
/// The default values are false because we need to be compatible with earlier access configurations /// The default values are false because we need to be compatible with earlier access configurations
@ -543,6 +651,26 @@ bool AccessControl::isPlaintextPasswordAllowed() const
return allow_plaintext_password; return allow_plaintext_password;
} }
void AccessControl::setPasswordComplexityRulesFromConfig(const Poco::Util::AbstractConfiguration & config_)
{
password_rules->setPasswordComplexityRulesFromConfig(config_);
}
void AccessControl::setPasswordComplexityRules(const std::vector<std::pair<String, String>> & rules_)
{
password_rules->setPasswordComplexityRules(rules_);
}
void AccessControl::checkPasswordComplexityRules(const String & password_) const
{
password_rules->checkPasswordComplexityRules(password_);
}
std::vector<std::pair<String, String>> AccessControl::getPasswordComplexityRules() const
{
return password_rules->getPasswordComplexityRules();
}
std::shared_ptr<const ContextAccess> AccessControl::getContextAccess( std::shared_ptr<const ContextAccess> AccessControl::getContextAccess(
const UUID & user_id, const UUID & user_id,

View File

@ -147,6 +147,13 @@ public:
void setPlaintextPasswordAllowed(const bool allow_plaintext_password_); void setPlaintextPasswordAllowed(const bool allow_plaintext_password_);
bool isPlaintextPasswordAllowed() const; bool isPlaintextPasswordAllowed() const;
/// Check complexity requirements for plaintext passwords
void setPasswordComplexityRulesFromConfig(const Poco::Util::AbstractConfiguration & config_);
void setPasswordComplexityRules(const std::vector<std::pair<String, String>> & rules_);
void checkPasswordComplexityRules(const String & password_) const;
std::vector<std::pair<String, String>> getPasswordComplexityRules() const;
/// Enables logic that users without permissive row policies can still read rows using a SELECT query. /// Enables logic that users without permissive row policies can still read rows using a SELECT query.
/// For example, if there two users A, B and a row policy is defined only for A, then /// For example, if there two users A, B and a row policy is defined only for A, then
/// if this setting is true the user B will see all rows, and if this setting is false the user B will see no rows. /// if this setting is true the user B will see all rows, and if this setting is false the user B will see no rows.
@ -212,6 +219,7 @@ public:
private: private:
class ContextAccessCache; class ContextAccessCache;
class CustomSettingsPrefixes; class CustomSettingsPrefixes;
class PasswordComplexityRules;
std::optional<UUID> insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override; std::optional<UUID> insertImpl(const AccessEntityPtr & entity, bool replace_if_exists, bool throw_if_exists) override;
bool removeImpl(const UUID & id, bool throw_if_not_exists) override; bool removeImpl(const UUID & id, bool throw_if_not_exists) override;
@ -225,6 +233,7 @@ private:
std::unique_ptr<ExternalAuthenticators> external_authenticators; std::unique_ptr<ExternalAuthenticators> external_authenticators;
std::unique_ptr<CustomSettingsPrefixes> custom_settings_prefixes; std::unique_ptr<CustomSettingsPrefixes> custom_settings_prefixes;
std::unique_ptr<AccessChangesNotifier> changes_notifier; std::unique_ptr<AccessChangesNotifier> changes_notifier;
std::unique_ptr<PasswordComplexityRules> password_rules;
std::atomic_bool allow_plaintext_password = true; std::atomic_bool allow_plaintext_password = true;
std::atomic_bool allow_no_password = true; std::atomic_bool allow_no_password = true;
std::atomic_bool allow_implicit_no_password = true; std::atomic_bool allow_implicit_no_password = true;

View File

@ -22,6 +22,7 @@
#include <Core/Block.h> #include <Core/Block.h>
#include <Core/Protocol.h> #include <Core/Protocol.h>
#include <Formats/FormatFactory.h> #include <Formats/FormatFactory.h>
#include <Access/AccessControl.h>
#include "config_version.h" #include "config_version.h"
@ -43,6 +44,7 @@
#include <Parsers/ASTInsertQuery.h> #include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTCreateQuery.h> #include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTCreateFunctionQuery.h> #include <Parsers/ASTCreateFunctionQuery.h>
#include <Parsers/Access/ASTCreateUserQuery.h>
#include <Parsers/ASTDropQuery.h> #include <Parsers/ASTDropQuery.h>
#include <Parsers/ASTSetQuery.h> #include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTUseQuery.h> #include <Parsers/ASTUseQuery.h>
@ -1562,6 +1564,15 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
updateLoggerLevel(logs_level_field->safeGet<String>()); updateLoggerLevel(logs_level_field->safeGet<String>());
} }
if (const auto * create_user_query = parsed_query->as<ASTCreateUserQuery>())
{
if (!create_user_query->attach && create_user_query->temporary_password_for_checks)
{
global_context->getAccessControl().checkPasswordComplexityRules(create_user_query->temporary_password_for_checks.value());
create_user_query->temporary_password_for_checks.reset();
}
}
processed_rows = 0; processed_rows = 0;
written_first_block = false; written_first_block = false;
progress_indication.resetProgress(); progress_indication.resetProgress();

View File

@ -309,6 +309,21 @@ void Connection::receiveHello()
readVarUInt(server_version_patch, *in); readVarUInt(server_version_patch, *in);
else else
server_version_patch = server_revision; server_version_patch = server_revision;
if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES)
{
UInt64 rules_size;
readVarUInt(rules_size, *in);
password_complexity_rules.reserve(rules_size);
for (size_t i = 0; i < rules_size; ++i)
{
String original_pattern, exception_message;
readStringBinary(original_pattern, *in);
readStringBinary(exception_message, *in);
password_complexity_rules.push_back({std::move(original_pattern), std::move(exception_message)});
}
}
} }
else if (packet_type == Protocol::Server::Exception) else if (packet_type == Protocol::Server::Exception)
receiveException()->rethrow(); receiveException()->rethrow();

View File

@ -93,6 +93,8 @@ public:
Protocol::Compression getCompression() const { return compression; } Protocol::Compression getCompression() const { return compression; }
std::vector<std::pair<String, String>> getPasswordComplexityRules() const override { return password_complexity_rules; }
void sendQuery( void sendQuery(
const ConnectionTimeouts & timeouts, const ConnectionTimeouts & timeouts,
const String & query, const String & query,
@ -207,6 +209,8 @@ private:
*/ */
ThrottlerPtr throttler; ThrottlerPtr throttler;
std::vector<std::pair<String, String>> password_complexity_rules;
/// From where to read query execution result. /// From where to read query execution result.
std::shared_ptr<ReadBuffer> maybe_compressed_in; std::shared_ptr<ReadBuffer> maybe_compressed_in;
std::unique_ptr<NativeReader> block_in; std::unique_ptr<NativeReader> block_in;

View File

@ -82,6 +82,8 @@ public:
virtual const String & getDescription() const = 0; virtual const String & getDescription() const = 0;
virtual std::vector<std::pair<String, String>> getPasswordComplexityRules() const = 0;
/// If last flag is true, you need to call sendExternalTablesData after. /// If last flag is true, you need to call sendExternalTablesData after.
virtual void sendQuery( virtual void sendQuery(
const ConnectionTimeouts & timeouts, const ConnectionTimeouts & timeouts,

View File

@ -91,6 +91,8 @@ public:
const String & getDescription() const override { return description; } const String & getDescription() const override { return description; }
std::vector<std::pair<String, String>> getPasswordComplexityRules() const override { return {}; }
void sendQuery( void sendQuery(
const ConnectionTimeouts & timeouts, const ConnectionTimeouts & timeouts,
const String & query, const String & query,

View File

@ -52,7 +52,7 @@
/// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION,
/// later is just a number for server version (one number instead of commit SHA) /// later is just a number for server version (one number instead of commit SHA)
/// for simplicity (sometimes it may be more convenient in some use cases). /// for simplicity (sometimes it may be more convenient in some use cases).
#define DBMS_TCP_PROTOCOL_VERSION 54460 #define DBMS_TCP_PROTOCOL_VERSION 54461
#define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449
@ -68,3 +68,5 @@
/// The server will send query elapsed run time in the Progress packet. /// The server will send query elapsed run time in the Progress packet.
#define DBMS_MIN_PROTOCOL_VERSION_WITH_SERVER_QUERY_TIME_IN_PROGRESS 54460 #define DBMS_MIN_PROTOCOL_VERSION_WITH_SERVER_QUERY_TIME_IN_PROGRESS 54460
#define DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES 54461

View File

@ -108,6 +108,12 @@ BlockIO InterpreterCreateUserQuery::execute()
throw Exception(ErrorCodes::BAD_ARGUMENTS, throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Authentication type NO_PASSWORD must be explicitly specified, check the setting allow_implicit_no_password in the server configuration"); "Authentication type NO_PASSWORD must be explicitly specified, check the setting allow_implicit_no_password in the server configuration");
if (!query.attach && query.temporary_password_for_checks)
{
access_control.checkPasswordComplexityRules(query.temporary_password_for_checks.value());
query.temporary_password_for_checks.reset();
}
std::optional<RolesOrUsersSet> default_roles_from_query; std::optional<RolesOrUsersSet> default_roles_from_query;
if (query.default_roles) if (query.default_roles)
{ {

View File

@ -46,6 +46,8 @@ public:
std::optional<AuthenticationData> auth_data; std::optional<AuthenticationData> auth_data;
mutable std::optional<String> temporary_password_for_checks;
std::optional<AllowedClientHosts> hosts; std::optional<AllowedClientHosts> hosts;
std::optional<AllowedClientHosts> add_hosts; std::optional<AllowedClientHosts> add_hosts;
std::optional<AllowedClientHosts> remove_hosts; std::optional<AllowedClientHosts> remove_hosts;

View File

@ -51,7 +51,7 @@ namespace
} }
bool parseAuthenticationData(IParserBase::Pos & pos, Expected & expected, AuthenticationData & auth_data) bool parseAuthenticationData(IParserBase::Pos & pos, Expected & expected, AuthenticationData & auth_data, std::optional<String> & temporary_password_for_checks)
{ {
return IParserBase::wrapParseImpl(pos, [&] return IParserBase::wrapParseImpl(pos, [&]
{ {
@ -165,6 +165,10 @@ namespace
common_names.insert(ast_child->as<const ASTLiteral &>().value.safeGet<String>()); common_names.insert(ast_child->as<const ASTLiteral &>().value.safeGet<String>());
} }
/// Save password separately for future complexity rules check
if (expect_password)
temporary_password_for_checks = value;
auth_data = AuthenticationData{*type}; auth_data = AuthenticationData{*type};
if (auth_data.getType() == AuthenticationType::SHA256_PASSWORD) if (auth_data.getType() == AuthenticationType::SHA256_PASSWORD)
{ {
@ -438,6 +442,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
std::optional<String> new_name; std::optional<String> new_name;
std::optional<AuthenticationData> auth_data; std::optional<AuthenticationData> auth_data;
std::optional<String> temporary_password_for_checks;
std::optional<AllowedClientHosts> hosts; std::optional<AllowedClientHosts> hosts;
std::optional<AllowedClientHosts> add_hosts; std::optional<AllowedClientHosts> add_hosts;
std::optional<AllowedClientHosts> remove_hosts; std::optional<AllowedClientHosts> remove_hosts;
@ -452,9 +457,11 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (!auth_data) if (!auth_data)
{ {
AuthenticationData new_auth_data; AuthenticationData new_auth_data;
if (parseAuthenticationData(pos, expected, new_auth_data)) std::optional<String> new_temporary_password_for_checks;
if (parseAuthenticationData(pos, expected, new_auth_data, new_temporary_password_for_checks))
{ {
auth_data = std::move(new_auth_data); auth_data = std::move(new_auth_data);
temporary_password_for_checks = std::move(new_temporary_password_for_checks);
continue; continue;
} }
} }
@ -539,6 +546,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
query->names = std::move(names); query->names = std::move(names);
query->new_name = std::move(new_name); query->new_name = std::move(new_name);
query->auth_data = std::move(auth_data); query->auth_data = std::move(auth_data);
query->temporary_password_for_checks = std::move(temporary_password_for_checks);
query->hosts = std::move(hosts); query->hosts = std::move(hosts);
query->add_hosts = std::move(add_hosts); query->add_hosts = std::move(add_hosts);
query->remove_hosts = std::move(remove_hosts); query->remove_hosts = std::move(remove_hosts);

View File

@ -36,6 +36,7 @@
#include <Storages/MergeTree/MergeTreeDataPartUUID.h> #include <Storages/MergeTree/MergeTreeDataPartUUID.h>
#include <Storages/StorageS3Cluster.h> #include <Storages/StorageS3Cluster.h>
#include <Core/ExternalTable.h> #include <Core/ExternalTable.h>
#include <Access/AccessControl.h>
#include <Access/Credentials.h> #include <Access/Credentials.h>
#include <Storages/ColumnDefault.h> #include <Storages/ColumnDefault.h>
#include <DataTypes/DataTypeLowCardinality.h> #include <DataTypes/DataTypeLowCardinality.h>
@ -1193,6 +1194,17 @@ void TCPHandler::sendHello()
writeStringBinary(server_display_name, *out); writeStringBinary(server_display_name, *out);
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSION_PATCH) if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSION_PATCH)
writeVarUInt(DBMS_VERSION_PATCH, *out); writeVarUInt(DBMS_VERSION_PATCH, *out);
if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES)
{
auto rules = server.context()->getAccessControl().getPasswordComplexityRules();
writeVarUInt(rules.size(), *out);
for (const auto & [original_pattern, exception_message] : rules)
{
writeStringBinary(original_pattern, *out);
writeStringBinary(exception_message, *out);
}
}
out->next(); out->next();
} }

View File

@ -0,0 +1,25 @@
<clickhouse>
<password_complexity>
<rule>
<pattern>.{12}</pattern>
<message>be at least 12 characters long</message>
</rule>
<rule>
<pattern>\p{N}</pattern>
<message>contain at least 1 numeric character</message>
</rule>
<rule>
<pattern>\p{Ll}</pattern>
<message>contain at least 1 lowercase character</message>
</rule>
<rule>
<pattern>\p{Lu}</pattern>
<message>contain at least 1 uppercase character</message>
</rule>
<rule>
<pattern>[^\p{L}\p{N}]</pattern>
<message>contain at least 1 special character</message>
</rule>
</password_complexity>
</clickhouse>

View File

@ -0,0 +1,42 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance("node", main_configs=["configs/complexity_rules.xml"])
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_complexity_rules(start_cluster):
error_message = "DB::Exception: Invalid password. The password should: be at least 12 characters long, contain at least 1 numeric character, contain at least 1 lowercase character, contain at least 1 uppercase character, contain at least 1 special character"
assert error_message in node.query_and_get_error(
"CREATE USER u_1 IDENTIFIED WITH plaintext_password BY ''"
)
error_message = "DB::Exception: Invalid password. The password should: contain at least 1 lowercase character, contain at least 1 uppercase character, contain at least 1 special character"
assert error_message in node.query_and_get_error(
"CREATE USER u_2 IDENTIFIED WITH sha256_password BY '000000000000'"
)
error_message = "DB::Exception: Invalid password. The password should: contain at least 1 uppercase character, contain at least 1 special character"
assert error_message in node.query_and_get_error(
"CREATE USER u_3 IDENTIFIED WITH double_sha1_password BY 'a00000000000'"
)
error_message = "DB::Exception: Invalid password. The password should: contain at least 1 special character"
assert error_message in node.query_and_get_error(
"CREATE USER u_4 IDENTIFIED WITH plaintext_password BY 'aA0000000000'"
)
node.query("CREATE USER u_5 IDENTIFIED WITH plaintext_password BY 'aA!000000000'")
node.query("DROP USER u_5")