mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 10:02:01 +00:00
Merge pull request #55253 from vitlibar/fix-async-insert-access-check-for-defaults
Evaluate defaults during async insert safer
This commit is contained in:
commit
e55a615f06
@ -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;
|
||||||
|
@ -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;
|
||||||
|
|
||||||
|
@ -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
|
||||||
{
|
{
|
||||||
|
@ -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.
|
||||||
|
@ -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;
|
||||||
|
@ -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
|
||||||
|
@ -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);
|
||||||
|
@ -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'"
|
||||||
|
@ -0,0 +1,6 @@
|
|||||||
|
1 value_1
|
||||||
|
2 value_2
|
||||||
|
Not enough privileges
|
||||||
|
Not enough privileges
|
||||||
|
5 value_5
|
||||||
|
6 value_6
|
58
tests/queries/0_stateless/02885_async_insert_access_check_for_defaults.sh
Executable file
58
tests/queries/0_stateless/02885_async_insert_access_check_for_defaults.sh
Executable 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
|
Loading…
Reference in New Issue
Block a user