Merge pull request #55253 from vitlibar/fix-async-insert-access-check-for-defaults

Evaluate defaults during async insert safer
This commit is contained in:
alesapin 2023-10-07 12:26:35 +02:00 committed by GitHub
commit e55a615f06
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 116 additions and 15 deletions

View File

@ -4,6 +4,16 @@
namespace DB namespace DB
{ {
std::vector<UUID> EnabledRolesInfo::getCurrentRoles() const
{
return std::vector<UUID>{current_roles.begin(), current_roles.end()};
}
std::vector<UUID> EnabledRolesInfo::getEnabledRoles() const
{
return std::vector<UUID>{enabled_roles.begin(), enabled_roles.end()};
}
Strings EnabledRolesInfo::getCurrentRolesNames() const Strings EnabledRolesInfo::getCurrentRolesNames() const
{ {
Strings result; Strings result;

View File

@ -20,6 +20,9 @@ struct EnabledRolesInfo
AccessRights access; AccessRights access;
SettingsProfileElements settings_from_enabled_roles; SettingsProfileElements settings_from_enabled_roles;
std::vector<UUID> getCurrentRoles() const;
std::vector<UUID> getEnabledRoles() const;
Strings getCurrentRolesNames() const; Strings getCurrentRolesNames() const;
Strings getEnabledRolesNames() const; Strings getEnabledRolesNames() const;

View File

@ -14,6 +14,8 @@
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <boost/container/flat_set.hpp>
namespace DB namespace DB
{ {

View File

@ -60,10 +60,12 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS; extern const int BAD_ARGUMENTS;
} }
AsynchronousInsertQueue::InsertQuery::InsertQuery(const ASTPtr & query_, const Settings & settings_) AsynchronousInsertQueue::InsertQuery::InsertQuery(const ASTPtr & query_, const Settings & settings_, const std::optional<UUID> & user_id_, const std::vector<UUID> & current_roles_)
: query(query_->clone()) : query(query_->clone())
, query_str(queryToString(query)) , query_str(queryToString(query))
, settings(settings_) , settings(settings_)
, user_id(user_id_)
, current_roles(current_roles_)
, hash(calculateHash()) , hash(calculateHash())
{ {
} }
@ -72,6 +74,8 @@ AsynchronousInsertQueue::InsertQuery::InsertQuery(const InsertQuery & other)
: query(other.query->clone()) : query(other.query->clone())
, query_str(other.query_str) , query_str(other.query_str)
, settings(other.settings) , settings(other.settings)
, user_id(other.user_id)
, current_roles(other.current_roles)
, hash(other.hash) , hash(other.hash)
{ {
} }
@ -83,6 +87,8 @@ AsynchronousInsertQueue::InsertQuery::operator=(const InsertQuery & other)
{ {
query = other.query->clone(); query = other.query->clone();
query_str = other.query_str; query_str = other.query_str;
user_id = other.user_id;
current_roles = other.current_roles;
settings = other.settings; settings = other.settings;
hash = other.hash; hash = other.hash;
} }
@ -95,6 +101,13 @@ UInt128 AsynchronousInsertQueue::InsertQuery::calculateHash() const
SipHash siphash; SipHash siphash;
query->updateTreeHash(siphash); query->updateTreeHash(siphash);
if (user_id)
{
siphash.update(*user_id);
for (const auto & current_role : current_roles)
siphash.update(current_role);
}
for (const auto & setting : settings.allChanged()) for (const auto & setting : settings.allChanged())
{ {
/// We don't consider this setting because it is only for deduplication, /// We don't consider this setting because it is only for deduplication,
@ -110,7 +123,7 @@ UInt128 AsynchronousInsertQueue::InsertQuery::calculateHash() const
bool AsynchronousInsertQueue::InsertQuery::operator==(const InsertQuery & other) const bool AsynchronousInsertQueue::InsertQuery::operator==(const InsertQuery & other) const
{ {
return query_str == other.query_str && settings == other.settings; return query_str == other.query_str && user_id == other.user_id && current_roles == other.current_roles && settings == other.settings;
} }
AsynchronousInsertQueue::InsertData::Entry::Entry(String && bytes_, String && query_id_, const String & async_dedup_token_, MemoryTracker * user_memory_tracker_) AsynchronousInsertQueue::InsertData::Entry::Entry(String && bytes_, String && query_id_, const String & async_dedup_token_, MemoryTracker * user_memory_tracker_)
@ -257,12 +270,9 @@ AsynchronousInsertQueue::push(ASTPtr query, ContextPtr query_context)
} }
} }
if (auto quota = query_context->getQuota())
quota->used(QuotaType::WRITTEN_BYTES, bytes.size());
auto entry = std::make_shared<InsertData::Entry>(std::move(bytes), query_context->getCurrentQueryId(), settings.insert_deduplication_token, CurrentThread::getUserMemoryTracker()); auto entry = std::make_shared<InsertData::Entry>(std::move(bytes), query_context->getCurrentQueryId(), settings.insert_deduplication_token, CurrentThread::getUserMemoryTracker());
InsertQuery key{query, settings}; InsertQuery key{query, settings, query_context->getUserID(), query_context->getCurrentRoles()};
InsertDataPtr data_to_process; InsertDataPtr data_to_process;
std::future<void> insert_future; std::future<void> insert_future;
@ -469,6 +479,11 @@ try
/// 'resetParser' doesn't work for parallel parsing. /// 'resetParser' doesn't work for parallel parsing.
key.settings.set("input_format_parallel_parsing", false); key.settings.set("input_format_parallel_parsing", false);
insert_context->makeQueryContext(); insert_context->makeQueryContext();
/// Access rights must be checked for the user who executed the initial INSERT query.
if (key.user_id)
insert_context->setUser(*key.user_id, key.current_roles);
insert_context->setSettings(key.settings); insert_context->setSettings(key.settings);
/// Set initial_query_id, because it's used in InterpreterInsertQuery for table lock. /// Set initial_query_id, because it's used in InterpreterInsertQuery for table lock.

View File

@ -53,9 +53,11 @@ private:
ASTPtr query; ASTPtr query;
String query_str; String query_str;
Settings settings; Settings settings;
std::optional<UUID> user_id;
std::vector<UUID> current_roles;
UInt128 hash; UInt128 hash;
InsertQuery(const ASTPtr & query_, const Settings & settings_); InsertQuery(const ASTPtr & query_, const Settings & settings_, const std::optional<UUID> & user_id_, const std::vector<UUID> & current_roles_);
InsertQuery(const InsertQuery & other); InsertQuery(const InsertQuery & other);
InsertQuery & operator=(const InsertQuery & other); InsertQuery & operator=(const InsertQuery & other);
bool operator==(const InsertQuery & other) const; bool operator==(const InsertQuery & other) const;

View File

@ -1209,14 +1209,14 @@ void Context::setCurrentRolesDefault()
setCurrentRoles(user->granted_roles.findGranted(user->default_roles)); setCurrentRoles(user->granted_roles.findGranted(user->default_roles));
} }
boost::container::flat_set<UUID> Context::getCurrentRoles() const std::vector<UUID> Context::getCurrentRoles() const
{ {
return getRolesInfo()->current_roles; return getRolesInfo()->getCurrentRoles();
} }
boost::container::flat_set<UUID> Context::getEnabledRoles() const std::vector<UUID> Context::getEnabledRoles() const
{ {
return getRolesInfo()->enabled_roles; return getRolesInfo()->getEnabledRoles();
} }
std::shared_ptr<const EnabledRolesInfo> Context::getRolesInfo() const std::shared_ptr<const EnabledRolesInfo> Context::getRolesInfo() const

View File

@ -28,7 +28,6 @@
#include "config.h" #include "config.h"
#include <boost/container/flat_set.hpp>
#include <functional> #include <functional>
#include <memory> #include <memory>
#include <mutex> #include <mutex>
@ -571,8 +570,8 @@ public:
void setCurrentRoles(const std::vector<UUID> & current_roles_); void setCurrentRoles(const std::vector<UUID> & current_roles_);
void setCurrentRolesDefault(); void setCurrentRolesDefault();
boost::container::flat_set<UUID> getCurrentRoles() const; std::vector<UUID> getCurrentRoles() const;
boost::container::flat_set<UUID> getEnabledRoles() const; std::vector<UUID> getEnabledRoles() const;
std::shared_ptr<const EnabledRolesInfo> getRolesInfo() const; std::shared_ptr<const EnabledRolesInfo> getRolesInfo() const;
void setCurrentProfile(const String & profile_name, bool check_constraints = true); void setCurrentProfile(const String & profile_name, bool check_constraints = true);

View File

@ -16,10 +16,13 @@ ${CLICKHOUSE_CLIENT} -q "CREATE ROLE r02247"
${CLICKHOUSE_CLIENT} -q "CREATE USER u02247" ${CLICKHOUSE_CLIENT} -q "CREATE USER u02247"
${CLICKHOUSE_CLIENT} -q "GRANT ALL ON *.* TO r02247" ${CLICKHOUSE_CLIENT} -q "GRANT ALL ON *.* TO r02247"
${CLICKHOUSE_CLIENT} -q "GRANT r02247 to u02247" ${CLICKHOUSE_CLIENT} -q "GRANT r02247 to u02247"
${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02247 FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 25 TO r02247" ${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02247 FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 30 TO r02247"
# The value 'qwqw' means about 13 bytes are to be written, so the current quota (30 bytes) gives the ability to write 'qwqw' 2 times.
${CLICKHOUSE_CLIENT} --user u02247 --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" ${CLICKHOUSE_CLIENT} --user u02247 --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')"
#${CLICKHOUSE_CLIENT} --user u02247 -q "SHOW CURRENT QUOTA"
${CLICKHOUSE_CLIENT} --user u02247 --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" ${CLICKHOUSE_CLIENT} --user u02247 --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')"
#${CLICKHOUSE_CLIENT} --user u02247 -q "SHOW CURRENT QUOTA"
${CLICKHOUSE_CLIENT} --user u02247 --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED ${CLICKHOUSE_CLIENT} --user u02247 --async_insert 1 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED
${CLICKHOUSE_CLIENT} --user u02247 --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED ${CLICKHOUSE_CLIENT} --user u02247 --async_insert 0 -q "INSERT INTO written_bytes_02247 VALUES ('qwqw')" 2>&1 | grep -m1 -o QUOTA_EXCEEDED
@ -30,7 +33,10 @@ ${CLICKHOUSE_CLIENT} -q "DROP QUOTA q02247"
${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02247 FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 1000 TO r02247" ${CLICKHOUSE_CLIENT} -q "CREATE QUOTA q02247 FOR INTERVAL 100 YEAR MAX WRITTEN BYTES = 1000 TO r02247"
${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE written_bytes_02247" ${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE written_bytes_02247"
# Numbers from 0 to 50 means about 540 bytes are to be written, so the current quota (1000 bytes) is enough to do so.
${CLICKHOUSE_CLIENT} --user u02247 -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(50)" ${CLICKHOUSE_CLIENT} --user u02247 -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(50)"
# Numbers from 0 to 100 means about 1090 bytes are to be written, so the current quota (1000 bytes total - 540 bytes already used) is NOT enough to do so.
${CLICKHOUSE_CLIENT} --user u02247 -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(100)" 2>&1 | grep -m1 -o QUOTA_EXCEEDED ${CLICKHOUSE_CLIENT} --user u02247 -q "INSERT INTO written_bytes_02247 SELECT toString(number) FROM numbers(100)" 2>&1 | grep -m1 -o QUOTA_EXCEEDED
${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 100 FROM system.quotas_usage WHERE quota_name = 'q02247'" ${CLICKHOUSE_CLIENT} -q "SELECT written_bytes > 100 FROM system.quotas_usage WHERE quota_name = 'q02247'"

View File

@ -0,0 +1,6 @@
1 value_1
2 value_2
Not enough privileges
Not enough privileges
5 value_5
6 value_6

View File

@ -0,0 +1,58 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CURDIR"/../shell_config.sh
restricted_user=${CLICKHOUSE_DATABASE}_restricted_user_$RANDOM$RANDOM
# The 'restricted_user' will not have access to the dictionary 'dict',
# so they shouldn't be able to insert to a table with DEFAULT dictGet(dict, ...)
$CLICKHOUSE_CLIENT --multiquery <<EOF
DROP USER IF EXISTS ${restricted_user};
DROP TABLE IF EXISTS table_with_default;
DROP DICTIONARY IF EXISTS dict;
DROP TABLE IF EXISTS table_for_dict;
CREATE USER ${restricted_user};
CREATE TABLE table_for_dict (key UInt64, value String) ENGINE = Memory();
INSERT INTO table_for_dict VALUES (1, 'value_1'), (2, 'value_2'), (3, 'value_3'), (4, 'value_4'), (5, 'value_5'), (6, 'value_6');
CREATE DICTIONARY dict
(
key UInt64,
value String
)
PRIMARY KEY key
SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict' DB currentDatabase()))
LIFETIME(1)
LAYOUT(FLAT());
CREATE TABLE table_with_default(key UInt64, value String DEFAULT dictGetString(dict, 'value', key)) ENGINE = Memory();
GRANT INSERT, SELECT ON table_with_default TO ${restricted_user};
EOF
$CLICKHOUSE_CLIENT --query "INSERT INTO table_with_default (key) VALUES (1)"
$CLICKHOUSE_CLIENT --async_insert=1 --query "INSERT INTO table_with_default (key) VALUES (2)"
$CLICKHOUSE_CLIENT --query "SELECT * FROM table_with_default WHERE key IN [1, 2] ORDER BY key"
$CLICKHOUSE_CLIENT --user "${restricted_user}" --query "INSERT INTO table_with_default (key) VALUES (3)" 2>&1 | grep -m 1 -oF "Not enough privileges"
$CLICKHOUSE_CLIENT --user "${restricted_user}" --async_insert=1 --query "INSERT INTO table_with_default (key) VALUES (4)" 2>&1 | grep -m 1 -oF 'Not enough privileges'
$CLICKHOUSE_CLIENT --query "SELECT * FROM table_with_default WHERE key IN [3, 4] ORDER BY key"
# We give the 'restricted_user' access to the dictionary 'dict',
# so now they should be able to insert to a table with DEFAULT dictGet(dict, ...)
$CLICKHOUSE_CLIENT --query "GRANT dictGet ON dict TO ${restricted_user}"
$CLICKHOUSE_CLIENT --user "${restricted_user}" --query "INSERT INTO table_with_default (key) VALUES (5)"
$CLICKHOUSE_CLIENT --user "${restricted_user}" --async_insert=1 --query "INSERT INTO table_with_default (key) VALUES (6)"
$CLICKHOUSE_CLIENT --query "SELECT * FROM table_with_default WHERE key IN [5, 6] ORDER BY key"
$CLICKHOUSE_CLIENT --multiquery <<EOF
DROP USER ${restricted_user};
DROP TABLE table_with_default;
DROP DICTIONARY dict;
DROP TABLE table_for_dict;
EOF