Better solution

This commit is contained in:
Nikolay Degterinsky 2022-11-22 02:41:23 +00:00
parent 6825d85daa
commit 3fd10e0011
15 changed files with 319 additions and 59 deletions

View File

@ -16,6 +16,8 @@
#include <base/find_symbols.h>
#include <Access/AccessControl.h>
#include "config_version.h"
#include <Common/Exception.h>
#include <Common/formatReadable.h>
@ -180,6 +182,64 @@ std::vector<String> Client::loadWarningMessages()
}
/// Make query to get all server password complexity rules
void Client::loadPasswordComplexityRules()
{
/// Older server versions cannot execute the query loading password complexity rules.
constexpr UInt64 min_server_revision_to_load_warnings = DBMS_MIN_PROTOCOL_VERSION_WITH_VIEW_IF_PERMITTED;
if (server_revision < min_server_revision_to_load_warnings)
return;
connection->sendQuery(connection_parameters.timeouts,
"SELECT * FROM viewIfPermitted(SELECT pattern, message FROM system.password_rules ELSE null('pattern String, message String'))",
{} /* query_parameters */,
"" /* query_id */,
QueryProcessingStage::Complete,
&global_context->getSettingsRef(),
&global_context->getClientInfo(), false, {});
while (true)
{
Packet packet = connection->receivePacket();
switch (packet.type)
{
case Protocol::Server::Data:
if (packet.block)
{
const ColumnString & column_pattern = typeid_cast<const ColumnString &>(*packet.block.getByPosition(0).column);
const ColumnString & column_message = typeid_cast<const ColumnString &>(*packet.block.getByPosition(1).column);
auto & access_control = global_context->getAccessControl();
size_t rows = packet.block.rows();
for (size_t i = 0; i < rows; ++i)
access_control.addPasswordComplexityRule({column_pattern[i].get<String>(), column_message[i].get<String>()});
}
continue;
case Protocol::Server::Progress:
case Protocol::Server::ProfileInfo:
case Protocol::Server::Totals:
case Protocol::Server::Extremes:
case Protocol::Server::Log:
continue;
case Protocol::Server::Exception:
packet.exception->rethrow();
return;
case Protocol::Server::EndOfStream:
return;
case Protocol::Server::ProfileEvents:
continue;
default:
throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from server {}",
packet.type, connection->getDescription());
}
}
}
void Client::initialize(Poco::Util::Application & self)
{
Poco::Util::Application::initialize(self);
@ -258,6 +318,16 @@ try
if (is_interactive && !config().has("no-warnings"))
showWarnings();
/// Load password complexity rules
try
{
loadPasswordComplexityRules();
}
catch (...)
{
/// Ignore exception
}
if (is_interactive && !delayed_interactive)
{
runInteractive();

View File

@ -48,5 +48,6 @@ private:
void printChangedSettings() const;
void showWarnings();
std::vector<String> loadWarningMessages();
void loadPasswordComplexityRules();
};
}

View File

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

View File

@ -145,7 +145,7 @@ private:
class AccessControl::PasswordComplexityRules
{
public:
void setPasswordComplexityRules(const Poco::Util::AbstractConfiguration & config_)
void setPasswordComplexityRulesFromConfig(const Poco::Util::AbstractConfiguration & config_)
{
std::lock_guard lock{mutex};
@ -160,37 +160,49 @@ public:
{
if (key == "rule" || key.starts_with("rule["))
{
String match(config_.getString("password_complexity." + key + ".match"));
String pattern(config_.getString("password_complexity." + key + ".pattern"));
String message(config_.getString("password_complexity." + key + ".message"));
auto matcher = std::make_unique<RE2>(match, RE2::Quiet);
auto matcher = std::make_unique<RE2>(pattern, RE2::Quiet);
if (!matcher->ok())
throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP,
"Password complexity pattern {} cannot be compiled: {}",
match, matcher->error());
pattern, matcher->error());
rules.push_back({std::move(matcher), std::move(message)});
rules.push_back({std::move(matcher), std::move(pattern), std::move(message)});
}
}
}
}
void addPasswordComplexityRule(std::pair<String, String> rule_)
{
auto matcher = std::make_unique<RE2>(rule_.first, RE2::Quiet);
if (!matcher->ok())
throw Exception(ErrorCodes::CANNOT_COMPILE_REGEXP,
"Password complexity pattern {} cannot be compiled: {}",
rule_.first, matcher->error());
std::lock_guard lock{mutex};
rules.push_back({std::move(matcher), std::move(rule_.first), std::move(rule_.second)});
}
void checkPasswordComplexityRules(const String & password_) const
{
String exception_text;
bool failed = false;
std::lock_guard lock{mutex};
for (const auto & [matcher, message] : rules)
for (const auto & rule : rules)
{
if (!RE2::PartialMatch(password_, *matcher))
if (!RE2::PartialMatch(password_, *rule.matcher))
{
failed = true;
if (!exception_text.empty())
exception_text += ", ";
exception_text += message;
exception_text += rule.exception_message;
}
}
@ -198,8 +210,30 @@ public:
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:
std::vector<std::pair<std::unique_ptr<RE2>, String>> rules TSA_GUARDED_BY(mutex);
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;
};
@ -231,7 +265,7 @@ void AccessControl::setUpFromMainConfig(const Poco::Util::AbstractConfiguration
setImplicitNoPasswordAllowed(config_.getBool("allow_implicit_no_password", true));
setNoPasswordAllowed(config_.getBool("allow_no_password", true));
setPlaintextPasswordAllowed(config_.getBool("allow_plaintext_password", true));
setPasswordComplexityRules(config_);
setPasswordComplexityRulesFromConfig(config_);
/// Optional improvements in access control system.
/// The default values are false because we need to be compatible with earlier access configurations
@ -597,9 +631,14 @@ bool AccessControl::isPlaintextPasswordAllowed() const
return allow_plaintext_password;
}
void AccessControl::setPasswordComplexityRules(const Poco::Util::AbstractConfiguration & config_)
void AccessControl::setPasswordComplexityRulesFromConfig(const Poco::Util::AbstractConfiguration & config_)
{
password_rules->setPasswordComplexityRules(config_);
password_rules->setPasswordComplexityRulesFromConfig(config_);
}
void AccessControl::addPasswordComplexityRule(std::pair<String, String> rule_)
{
password_rules->addPasswordComplexityRule(std::move(rule_));
}
void AccessControl::checkPasswordComplexityRules(const String & password_) const
@ -607,6 +646,11 @@ 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(
const UUID & user_id,

View File

@ -148,8 +148,11 @@ public:
bool isPlaintextPasswordAllowed() const;
/// Check complexity requirements for plaintext passwords
void setPasswordComplexityRules(const Poco::Util::AbstractConfiguration & config_);
void setPasswordComplexityRulesFromConfig(const Poco::Util::AbstractConfiguration & config_);
void addPasswordComplexityRule(std::pair<String, String> rule_);
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.
/// For example, if there two users A, B and a row policy is defined only for A, then

View File

@ -22,6 +22,7 @@
#include <Core/Block.h>
#include <Core/Protocol.h>
#include <Formats/FormatFactory.h>
#include <Access/AccessControl.h>
#include "config_version.h"
@ -43,6 +44,7 @@
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTCreateFunctionQuery.h>
#include <Parsers/Access/ASTCreateUserQuery.h>
#include <Parsers/ASTDropQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTUseQuery.h>
@ -1563,6 +1565,15 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
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;
written_first_block = false;
progress_indication.resetProgress();

View File

@ -108,8 +108,11 @@ BlockIO InterpreterCreateUserQuery::execute()
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Authentication type NO_PASSWORD must be explicitly specified, check the setting allow_implicit_no_password in the server configuration");
if (query.auth_data && query.auth_data->getType() == AuthenticationType::PLAINTEXT_PASSWORD)
access_control.checkPasswordComplexityRules(query.auth_data->getPassword());
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;
if (query.default_roles)

View File

@ -47,6 +47,8 @@ public:
std::optional<AuthenticationData> auth_data;
bool show_password = true; /// formatImpl() shows a password or hash by default
mutable std::optional<String> temporary_password_for_checks;
std::optional<AllowedClientHosts> hosts;
std::optional<AllowedClientHosts> add_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, [&]
{
@ -165,6 +165,10 @@ namespace
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};
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<AuthenticationData> auth_data;
std::optional<String> temporary_password_for_checks;
std::optional<AllowedClientHosts> hosts;
std::optional<AllowedClientHosts> add_hosts;
std::optional<AllowedClientHosts> remove_hosts;
@ -452,9 +457,11 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (!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);
temporary_password_for_checks = std::move(new_temporary_password_for_checks);
continue;
}
}
@ -539,6 +546,7 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
query->names = std::move(names);
query->new_name = std::move(new_name);
query->auth_data = std::move(auth_data);
query->temporary_password_for_checks = std::move(temporary_password_for_checks);
query->hosts = std::move(hosts);
query->add_hosts = std::move(add_hosts);
query->remove_hosts = std::move(remove_hosts);

View File

@ -0,0 +1,26 @@
#include <Interpreters/Context.h>
#include <Access/AccessControl.h>
#include <Storages/System/StorageSystemPasswordRules.h>
namespace DB
{
NamesAndTypesList StorageSystemPasswordRules::getNamesAndTypes()
{
return {
{"pattern", std::make_shared<DataTypeString>()},
{"message", std::make_shared<DataTypeString>()},
};
}
void StorageSystemPasswordRules::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
for (const auto & [original_pattern, exception_message] : context->getAccessControl().getPasswordComplexityRules())
{
res_columns[0]->insert(original_pattern);
res_columns[1]->insert(exception_message);
}
}
}

View File

@ -0,0 +1,26 @@
#pragma once
#include <Storages/System/IStorageSystemOneBlock.h>
namespace DB
{
class Context;
/** Implements system.password_rules table that contains complexity rules for passwords for users
* to be applied in clickhouse-client.
*/
class StorageSystemPasswordRules final : public IStorageSystemOneBlock<StorageSystemPasswordRules>
{
public:
std::string getName() const override { return "SystemPasswordRules"; }
static NamesAndTypesList getNamesAndTypes();
protected:
using IStorageSystemOneBlock::IStorageSystemOneBlock;
void fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const override;
};
}

View File

@ -31,6 +31,7 @@
#include <Storages/System/StorageSystemOne.h>
#include <Storages/System/StorageSystemPartMovesBetweenShards.h>
#include <Storages/System/StorageSystemParts.h>
#include <Storages/System/StorageSystemPasswordRules.h>
#include <Storages/System/StorageSystemProjectionParts.h>
#include <Storages/System/StorageSystemPartsColumns.h>
#include <Storages/System/StorageSystemProjectionPartsColumns.h>
@ -131,6 +132,7 @@ void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database)
attach<StorageSystemPrivileges>(context, system_database, "privileges");
attach<StorageSystemErrors>(context, system_database, "errors");
attach<StorageSystemWarnings>(context, system_database, "warnings");
attach<StorageSystemPasswordRules>(context, system_database, "password_rules");
attach<StorageSystemDataSkippingIndices>(context, system_database, "data_skipping_indices");
attach<StorageSystemLicenses>(context, system_database, "licenses");
attach<StorageSystemTimeZones>(context, system_database, "time_zones");

View File

@ -5,41 +5,50 @@
</logger>
<tcp_port>9000</tcp_port>
<path>./</path>
<mark_cache_size>0</mark_cache_size>
<users>
<default>
<password></password>
<networks>
<ip>::/0</ip>
</networks>
<access_management>1</access_management>
</default>
</users>
<password_complexity>
<rule>
<match>.{12}</match>
<pattern>.{12}</pattern>
<message>be at least 12 characters long</message>
</rule>
<rule>
<match>\p{N}</match>
<pattern>\p{N}</pattern>
<message>contain at least 1 numeric character</message>
</rule>
<rule>
<match>\p{Ll}</match>
<pattern>\p{Ll}</pattern>
<message>contain at least 1 lowercase character</message>
</rule>
<rule>
<match>\p{Lu}</match>
<pattern>\p{Lu}</pattern>
<message>contain at least 1 uppercase character</message>
</rule>
<rule>
<match>[^\p{L}\p{N}]</match>
<pattern>[^\p{L}\p{N}]</pattern>
<message>contain at least 1 special character</message>
</rule>
</password_complexity>
<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>
<profiles>
<default/>
</profiles>
<quotas>
<default />
</quotas>
</clickhouse>

View File

@ -2,4 +2,3 @@ OK
OK
OK
OK
OK

View File

@ -1,27 +1,83 @@
#!/usr/bin/env bash
# Tags: no-parallel
# Tags: long, no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "DROP USER IF EXISTS u_2474"
server_opts=(
"--config-file=$CUR_DIR/$(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.log &
server_pid=$!
trap cleanup EXIT
function cleanup()
{
kill -9 $server_pid
kill -9 $client_pid
echo "Test failed. Server log:"
cat clickhouse-server.log
rm -f clickhouse-server.log
exit 1
}
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
done
if [[ -z $server_port ]]; then
echo "Cannot wait for LISTEN socket" >&2
exit 1
fi
# wait for the server to start accepting tcp connections (max 30 seconds)
i=0 retries=300
while ! $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null -q 'select 1' 2>/dev/null && [[ $i -lt $retries ]]; do
sleep 0.1
done
if ! $CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" --format Null -q 'select 1'; then
echo "Cannot wait until server will start accepting connections on <tcp_port>" >&2
exit 1
fi
$CLICKHOUSE_CLIENT -q "CREATE USER u_2474 IDENTIFIED WITH plaintext_password BY ''" 2>&1 | grep -qF \
"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." && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" -q "CREATE USER u_2474 IDENTIFIED WITH plaintext_password BY ''" 2>&1 | grep -qF \
"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" && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT -q "CREATE USER u_2474 IDENTIFIED WITH plaintext_password BY '000000000000'" 2>&1 | grep -qF \
"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." && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" -q "CREATE USER u_2474 IDENTIFIED WITH plaintext_password BY '000000000000'" 2>&1 | grep -qF \
"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" && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT -q "CREATE USER u_2474 IDENTIFIED WITH plaintext_password BY 'a00000000000'" 2>&1 | grep -qF \
"DB::Exception: Invalid password. The password should: contain at least 1 uppercase character, contain at least 1 special character." && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" -q "CREATE USER u_2474 IDENTIFIED WITH plaintext_password BY 'a00000000000'" 2>&1 | grep -qF \
"DB::Exception: Invalid password. The password should: contain at least 1 uppercase character, contain at least 1 special character" && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT -q "CREATE USER u_2474 IDENTIFIED WITH plaintext_password BY 'aA0000000000'" 2>&1 | grep -qF \
"DB::Exception: Invalid password. The password should: contain at least 1 special character." && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT -q "CREATE USER u_2474 IDENTIFIED WITH plaintext_password BY 'aA!000000000'" 2>&1 | grep -qF \
"DB::Exception:" && echo 'FAIL' || echo 'OK' ||:
$CLICKHOUSE_CLIENT_BINARY --host 127.1 --port "$server_port" -q "CREATE USER u_2474 IDENTIFIED WITH plaintext_password BY 'aA0000000000'" 2>&1 | grep -qF \
"DB::Exception: Invalid password. The password should: contain at least 1 special character" && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT -q "DROP USER u_2474"
# send TERM and save the error code to ensure that it is 0 (EXIT_SUCCESS)
kill $server_pid
wait $server_pid
return_code=$?
wait $client_pid
trap '' EXIT
if [ $return_code != 0 ]; then
cat clickhouse-server.log
fi
rm -f clickhouse-server.log
exit $return_code