Better implementation of password rules

This commit is contained in:
Nikolay Degterinsky 2022-12-11 23:47:43 +00:00
parent a8545a29a4
commit 19e95e8cd8
17 changed files with 59 additions and 135 deletions

View File

@ -182,64 +182,6 @@ 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_complexity_rules = DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES;
if (server_revision < min_server_revision_to_load_complexity_rules)
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);
@ -318,7 +260,9 @@ try
if (is_interactive && !config().has("no-warnings"))
showWarnings();
loadPasswordComplexityRules();
/// Set user password complexity rules
auto & access_control = global_context->getAccessControl();
access_control.setPasswordComplexityRules(connection->getPasswordComplexityRules());
if (is_interactive && !delayed_interactive)
{

View File

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

View File

@ -466,7 +466,7 @@
<allow_no_password>1</allow_no_password>
<allow_implicit_no_password>1</allow_implicit_no_password>
<!-- Complexity requirements for plaintext passwords. -->
<!-- Complexity requirements for user passwords. -->
<!-- <password_complexity>
<rule>
<pattern>.{12}</pattern>

View File

@ -175,16 +175,23 @@ public:
}
}
void addPasswordComplexityRule(std::pair<String, String> rule_)
void setPasswordComplexityRules(std::vector<std::pair<String, String>> rules_)
{
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());
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), std::move(original_pattern), std::move(exception_message)});
}
std::lock_guard lock{mutex};
rules.push_back({std::move(matcher), std::move(rule_.first), std::move(rule_.second)});
rules = std::move(new_rules);
}
void checkPasswordComplexityRules(const String & password_) const
@ -636,9 +643,9 @@ void AccessControl::setPasswordComplexityRulesFromConfig(const Poco::Util::Abstr
password_rules->setPasswordComplexityRulesFromConfig(config_);
}
void AccessControl::addPasswordComplexityRule(std::pair<String, String> rule_)
void AccessControl::setPasswordComplexityRules(std::vector<std::pair<String, String>> rules_)
{
password_rules->addPasswordComplexityRule(std::move(rule_));
password_rules->setPasswordComplexityRules(std::move(rules_));
}
void AccessControl::checkPasswordComplexityRules(const String & password_) const

View File

@ -150,7 +150,7 @@ public:
/// Check complexity requirements for plaintext passwords
void setPasswordComplexityRulesFromConfig(const Poco::Util::AbstractConfiguration & config_);
void addPasswordComplexityRule(std::pair<String, String> rule_);
void setPasswordComplexityRules(std::vector<std::pair<String, String>> rules_);
void checkPasswordComplexityRules(const String & password_) const;
std::vector<std::pair<String, String>> getPasswordComplexityRules() const;

View File

@ -308,6 +308,21 @@ void Connection::receiveHello()
readVarUInt(server_version_patch, *in);
else
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)
receiveException()->rethrow();

View File

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

View File

@ -82,6 +82,8 @@ public:
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.
virtual void sendQuery(
const ConnectionTimeouts & timeouts,

View File

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

View File

@ -69,4 +69,4 @@
/// 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_PASSWORD_COMPLEXITY_RULES 54460
#define DBMS_MIN_PROTOCOL_VERSION_WITH_PASSWORD_COMPLEXITY_RULES 54461

View File

@ -36,6 +36,7 @@
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
#include <Storages/StorageS3Cluster.h>
#include <Core/ExternalTable.h>
#include <Access/AccessControl.h>
#include <Access/Credentials.h>
#include <Storages/ColumnDefault.h>
#include <DataTypes/DataTypeLowCardinality.h>
@ -1193,6 +1194,17 @@ void TCPHandler::sendHello()
writeStringBinary(server_display_name, *out);
if (client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_VERSION_PATCH)
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();
}

View File

@ -1,26 +0,0 @@
#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

@ -1,26 +0,0 @@
#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

@ -32,7 +32,6 @@
#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>
@ -133,7 +132,6 @@ 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

@ -1,7 +1,3 @@
# pylint: disable=unused-argument
# pylint: disable=redefined-outer-name
# pylint: disable=line-too-long
import pytest
from helpers.cluster import ClickHouseCluster

View File

@ -9,16 +9,14 @@
{"initial query spans with proper parent":"1"}
{"unique non-empty tracestate values":"1"}
===native===
{"query":"SELECT * FROM viewIfPermitted(SELECT pattern, message FROM system.password_rules ELSE null('pattern String, message String'))","status":"QueryFinish","tracestate":"another custom state","sorted_by_start_time":1}
{"query":"select * from url('http:\/\/127.0.0.2:8123\/?query=select%201%20format%20Null', CSV, 'a int')","status":"QueryFinish","tracestate":"another custom state","sorted_by_start_time":1}
{"query":"select 1 format Null\n","status":"QueryFinish","tracestate":"another custom state","sorted_by_start_time":1}
{"query":"select 1 format Null\n","status":"QueryFinish","tracestate":"another custom state","sorted_by_start_time":1}
{"query":"SELECT * FROM viewIfPermitted(SELECT pattern, message FROM system.password_rules ELSE null('pattern String, message String'))","query_status":"QueryFinish","tracestate":"another custom state","sorted_by_finish_time":1}
{"query":"select 1 format Null\n","query_status":"QueryFinish","tracestate":"another custom state","sorted_by_finish_time":1}
{"query":"select 1 format Null\n","query_status":"QueryFinish","tracestate":"another custom state","sorted_by_finish_time":1}
{"query":"select * from url('http:\/\/127.0.0.2:8123\/?query=select%201%20format%20Null', CSV, 'a int')","query_status":"QueryFinish","tracestate":"another custom state","sorted_by_finish_time":1}
{"total spans":"4","unique spans":"4","unique non-zero parent spans":"4"}
{"initial query spans with proper parent":"2"}
{"total spans":"3","unique spans":"3","unique non-zero parent spans":"3"}
{"initial query spans with proper parent":"1"}
{"unique non-empty tracestate values":"1"}
===sampled===
OK

View File

@ -32,7 +32,6 @@ None
3
4
Check system.query_log
SELECT * FROM viewIfPermitted(SELECT pattern, message FROM system.password_rules ELSE null(\'pattern String, message String\')) []
SELECT \'None\'; []
SELECT * FROM 02131_rqtable; []
SELECT \'R1: x == 1\'; []