mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Better implementation of password rules
This commit is contained in:
parent
a8545a29a4
commit
19e95e8cd8
@ -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)
|
||||
{
|
||||
|
@ -48,6 +48,5 @@ private:
|
||||
void printChangedSettings() const;
|
||||
void showWarnings();
|
||||
std::vector<String> loadWarningMessages();
|
||||
void loadPasswordComplexityRules();
|
||||
};
|
||||
}
|
||||
|
@ -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>
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -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;
|
||||
};
|
||||
}
|
@ -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");
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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\'; []
|
||||
|
Loading…
Reference in New Issue
Block a user