Get rid of QueryCache::user_name

This commit is contained in:
Robert Schulze 2024-01-09 12:05:02 +00:00
parent bca0f144e2
commit e9b6f413b8
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
8 changed files with 69 additions and 54 deletions

View File

@ -5,15 +5,15 @@
#include <Common/LRUCachePolicy.h> #include <Common/LRUCachePolicy.h>
#include <Common/SLRUCachePolicy.h> #include <Common/SLRUCachePolicy.h>
#include <base/UUID.h>
#include <base/defines.h>
#include <atomic> #include <atomic>
#include <cassert>
#include <chrono>
#include <memory> #include <memory>
#include <mutex> #include <mutex>
#include <optional>
#include <unordered_map> #include <unordered_map>
#include <base/defines.h>
namespace DB namespace DB
{ {
@ -227,10 +227,10 @@ public:
cache_policy->setMaxSizeInBytes(max_size_in_bytes); cache_policy->setMaxSizeInBytes(max_size_in_bytes);
} }
void setQuotaForUser(const String & user_name, size_t max_size_in_bytes, size_t max_entries) void setQuotaForUser(std::optional<UUID> user_id, size_t max_size_in_bytes, size_t max_entries)
{ {
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
cache_policy->setQuotaForUser(user_name, max_size_in_bytes, max_entries); cache_policy->setQuotaForUser(user_id, max_size_in_bytes, max_entries);
} }
virtual ~CacheBase() = default; virtual ~CacheBase() = default;

View File

@ -2,10 +2,11 @@
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/ICachePolicyUserQuota.h> #include <Common/ICachePolicyUserQuota.h>
#include <base/UUID.h>
#include <functional> #include <functional>
#include <memory> #include <memory>
#include <mutex> #include <optional>
namespace DB namespace DB
{ {
@ -43,7 +44,7 @@ public:
virtual void setMaxCount(size_t /*max_count*/) = 0; virtual void setMaxCount(size_t /*max_count*/) = 0;
virtual void setMaxSizeInBytes(size_t /*max_size_in_bytes*/) = 0; virtual void setMaxSizeInBytes(size_t /*max_size_in_bytes*/) = 0;
virtual void setQuotaForUser(const String & user_name, size_t max_size_in_bytes, size_t max_entries) { user_quotas->setQuotaForUser(user_name, max_size_in_bytes, max_entries); } virtual void setQuotaForUser(std::optional<UUID> user_id, size_t max_size_in_bytes, size_t max_entries) { user_quotas->setQuotaForUser(user_id, max_size_in_bytes, max_entries); }
/// HashFunction usually hashes the entire key and the found key will be equal the provided key. In such cases, use get(). It is also /// HashFunction usually hashes the entire key and the found key will be equal the provided key. In such cases, use get(). It is also
/// possible to store other, non-hashed data in the key. In that case, the found key is potentially different from the provided key. /// possible to store other, non-hashed data in the key. In that case, the found key is potentially different from the provided key.

View File

@ -1,7 +1,10 @@
#pragma once #pragma once
#include <base/UUID.h>
#include <base/types.h> #include <base/types.h>
#include <optional>
namespace DB namespace DB
{ {
@ -15,14 +18,14 @@ class ICachePolicyUserQuota
{ {
public: public:
/// Register or update the user's quota for the given resource. /// Register or update the user's quota for the given resource.
virtual void setQuotaForUser(const String & user_name, size_t max_size_in_bytes, size_t max_entries) = 0; virtual void setQuotaForUser(std::optional<UUID> user_id, size_t max_size_in_bytes, size_t max_entries) = 0;
/// Update the actual resource usage for the given user. /// Update the actual resource usage for the given user.
virtual void increaseActual(const String & user_name, size_t entry_size_in_bytes) = 0; virtual void increaseActual(std::optional<UUID> user_id, size_t entry_size_in_bytes) = 0;
virtual void decreaseActual(const String & user_name, size_t entry_size_in_bytes) = 0; virtual void decreaseActual(std::optional<UUID> user_id, size_t entry_size_in_bytes) = 0;
/// Is the user allowed to write a new entry into the cache? /// Is the user allowed to write a new entry into the cache?
virtual bool approveWrite(const String & user_name, size_t entry_size_in_bytes) const = 0; virtual bool approveWrite(std::optional<UUID> user_id, size_t entry_size_in_bytes) const = 0;
virtual ~ICachePolicyUserQuota() = default; virtual ~ICachePolicyUserQuota() = default;
}; };
@ -33,10 +36,10 @@ using CachePolicyUserQuotaPtr = std::unique_ptr<ICachePolicyUserQuota>;
class NoCachePolicyUserQuota : public ICachePolicyUserQuota class NoCachePolicyUserQuota : public ICachePolicyUserQuota
{ {
public: public:
void setQuotaForUser(const String & /*user_name*/, size_t /*max_size_in_bytes*/, size_t /*max_entries*/) override {} void setQuotaForUser(std::optional<UUID> /*user_id*/, size_t /*max_size_in_bytes*/, size_t /*max_entries*/) override {}
void increaseActual(const String & /*user_name*/, size_t /*entry_size_in_bytes*/) override {} void increaseActual(std::optional<UUID> /*user_id*/, size_t /*entry_size_in_bytes*/) override {}
void decreaseActual(const String & /*user_name*/, size_t /*entry_size_in_bytes*/) override {} void decreaseActual(std::optional<UUID> /*user_id*/, size_t /*entry_size_in_bytes*/) override {}
bool approveWrite(const String & /*user_name*/, size_t /*entry_size_in_bytes*/) const override { return true; } bool approveWrite(std::optional<UUID> /*user_id*/, size_t /*entry_size_in_bytes*/) const override { return true; }
}; };

View File

@ -1,8 +1,10 @@
#pragma once #pragma once
#include <Common/ICachePolicy.h> #include <Common/ICachePolicy.h>
#include <base/UUID.h>
#include <limits> #include <limits>
#include <optional>
#include <unordered_map> #include <unordered_map>
namespace DB namespace DB
@ -11,37 +13,47 @@ namespace DB
class PerUserTTLCachePolicyUserQuota : public ICachePolicyUserQuota class PerUserTTLCachePolicyUserQuota : public ICachePolicyUserQuota
{ {
public: public:
void setQuotaForUser(const String & user_name, size_t max_size_in_bytes, size_t max_entries) override void setQuotaForUser(std::optional<UUID> user_id, size_t max_size_in_bytes, size_t max_entries) override
{ {
quotas[user_name] = {max_size_in_bytes, max_entries}; if (!user_id.has_value())
return;
quotas[*user_id] = {max_size_in_bytes, max_entries};
} }
void increaseActual(const String & user_name, size_t entry_size_in_bytes) override void increaseActual(std::optional<UUID> user_id, size_t entry_size_in_bytes) override
{ {
auto & actual_for_user = actual[user_name]; if (!user_id.has_value())
return;
auto & actual_for_user = actual[*user_id];
actual_for_user.size_in_bytes += entry_size_in_bytes; actual_for_user.size_in_bytes += entry_size_in_bytes;
actual_for_user.num_items += 1; actual_for_user.num_items += 1;
} }
void decreaseActual(const String & user_name, size_t entry_size_in_bytes) override void decreaseActual(std::optional<UUID> user_id, size_t entry_size_in_bytes) override
{ {
chassert(actual.contains(user_name)); if (!user_id.has_value())
return;
chassert(actual[user_name].size_in_bytes >= entry_size_in_bytes); chassert(actual.contains(*user_id));
actual[user_name].size_in_bytes -= entry_size_in_bytes;
chassert(actual[user_name].num_items >= 1); chassert(actual[*user_id].size_in_bytes >= entry_size_in_bytes);
actual[user_name].num_items -= 1; actual[*user_id].size_in_bytes -= entry_size_in_bytes;
chassert(actual[*user_id].num_items >= 1);
actual[*user_id].num_items -= 1;
} }
bool approveWrite(const String & user_name, size_t entry_size_in_bytes) const override bool approveWrite(std::optional<UUID> user_id, size_t entry_size_in_bytes) const override
{ {
auto it_actual = actual.find(user_name); if (!user_id.has_value())
return true;
auto it_actual = actual.find(*user_id);
Resources actual_for_user{.size_in_bytes = 0, .num_items = 0}; /// assume zero actual resource consumption is user isn't found Resources actual_for_user{.size_in_bytes = 0, .num_items = 0}; /// assume zero actual resource consumption is user isn't found
if (it_actual != actual.end()) if (it_actual != actual.end())
actual_for_user = it_actual->second; actual_for_user = it_actual->second;
auto it_quota = quotas.find(user_name); auto it_quota = quotas.find(*user_id);
Resources quota_for_user{.size_in_bytes = std::numeric_limits<size_t>::max(), .num_items = std::numeric_limits<size_t>::max()}; /// assume no threshold if no quota is found Resources quota_for_user{.size_in_bytes = std::numeric_limits<size_t>::max(), .num_items = std::numeric_limits<size_t>::max()}; /// assume no threshold if no quota is found
if (it_quota != quotas.end()) if (it_quota != quotas.end())
quota_for_user = it_quota->second; quota_for_user = it_quota->second;
@ -69,10 +81,10 @@ public:
size_t num_items = 0; size_t num_items = 0;
}; };
/// user name --> cache size quota (in bytes) / number of items quota /// user id --> cache size quota (in bytes) / number of items quota
std::map<String, Resources> quotas; std::map<UUID, Resources> quotas;
/// user name --> actual cache usage (in bytes) / number of items /// user id --> actual cache usage (in bytes) / number of items
std::map<String, Resources> actual; std::map<UUID, Resources> actual;
}; };
@ -132,7 +144,7 @@ public:
if (it == cache.end()) if (it == cache.end())
return; return;
size_t sz = weight_function(*it->second); size_t sz = weight_function(*it->second);
Base::user_quotas->decreaseActual(it->first.user_name, sz); Base::user_quotas->decreaseActual(it->first.user_id, sz);
cache.erase(it); cache.erase(it);
size_in_bytes -= sz; size_in_bytes -= sz;
} }
@ -169,7 +181,7 @@ public:
/// Checks against per-user limits /// Checks against per-user limits
auto sufficient_space_in_cache_for_user = [&]() auto sufficient_space_in_cache_for_user = [&]()
{ {
return Base::user_quotas->approveWrite(key.user_name, entry_size_in_bytes); return Base::user_quotas->approveWrite(key.user_id, entry_size_in_bytes);
}; };
if (!sufficient_space_in_cache() || !sufficient_space_in_cache_for_user()) if (!sufficient_space_in_cache() || !sufficient_space_in_cache_for_user())
@ -179,7 +191,7 @@ public:
if (is_stale_function(it->first)) if (is_stale_function(it->first))
{ {
size_t sz = weight_function(*it->second); size_t sz = weight_function(*it->second);
Base::user_quotas->decreaseActual(it->first.user_name, sz); Base::user_quotas->decreaseActual(it->first.user_id, sz);
it = cache.erase(it); it = cache.erase(it);
size_in_bytes -= sz; size_in_bytes -= sz;
} }
@ -193,14 +205,14 @@ public:
if (auto it = cache.find(key); it != cache.end()) if (auto it = cache.find(key); it != cache.end())
{ {
size_t sz = weight_function(*it->second); size_t sz = weight_function(*it->second);
Base::user_quotas->decreaseActual(it->first.user_name, sz); Base::user_quotas->decreaseActual(it->first.user_id, sz);
cache.erase(it); // stupid bug: (*) doesn't replace existing entries (likely due to custom hash function), need to erase explicitly cache.erase(it); // stupid bug: (*) doesn't replace existing entries (likely due to custom hash function), need to erase explicitly
size_in_bytes -= sz; size_in_bytes -= sz;
} }
cache[key] = std::move(mapped); // (*) cache[key] = std::move(mapped); // (*)
size_in_bytes += entry_size_in_bytes; size_in_bytes += entry_size_in_bytes;
Base::user_quotas->increaseActual(key.user_name, entry_size_in_bytes); Base::user_quotas->increaseActual(key.user_id, entry_size_in_bytes);
} }
} }

View File

@ -129,13 +129,12 @@ String queryStringFromAST(ASTPtr ast)
QueryCache::Key::Key( QueryCache::Key::Key(
ASTPtr ast_, ASTPtr ast_,
Block header_, Block header_,
const String & user_name_, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_,
bool is_shared_, bool is_shared_,
std::chrono::time_point<std::chrono::system_clock> expires_at_, std::chrono::time_point<std::chrono::system_clock> expires_at_,
bool is_compressed_) bool is_compressed_)
: ast(removeQueryCacheSettings(ast_)) : ast(removeQueryCacheSettings(ast_))
, header(header_) , header(header_)
, user_name(user_name_)
, user_id(user_id_) , user_id(user_id_)
, current_user_roles(current_user_roles_) , current_user_roles(current_user_roles_)
, is_shared(is_shared_) , is_shared(is_shared_)
@ -145,8 +144,8 @@ QueryCache::Key::Key(
{ {
} }
QueryCache::Key::Key(ASTPtr ast_, const String & user_name_, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_) QueryCache::Key::Key(ASTPtr ast_, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_)
: QueryCache::Key(ast_, {}, user_name_, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST or user name : QueryCache::Key(ast_, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST or user name
{ {
} }
@ -404,10 +403,9 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar
const auto & entry_key = entry->key; const auto & entry_key = entry->key;
const auto & entry_mapped = entry->mapped; const auto & entry_mapped = entry->mapped;
const bool is_same_user_name = (entry_key.user_name == key.user_name);
const bool is_same_user_id = ((!entry_key.user_id.has_value() && !key.user_id.has_value()) || (entry_key.user_id.has_value() && key.user_id.has_value() && *entry_key.user_id == *key.user_id)); const bool is_same_user_id = ((!entry_key.user_id.has_value() && !key.user_id.has_value()) || (entry_key.user_id.has_value() && key.user_id.has_value() && *entry_key.user_id == *key.user_id));
const bool is_same_current_user_roles = (entry_key.current_user_roles == key.current_user_roles); const bool is_same_current_user_roles = (entry_key.current_user_roles == key.current_user_roles);
if (!entry_key.is_shared && (!is_same_user_name || !is_same_user_id || !is_same_current_user_roles)) if (!entry_key.is_shared && (!is_same_user_id || !is_same_current_user_roles))
{ {
LOG_TRACE(logger, "Inaccessible query result found for query {}", doubleQuoteString(key.query_string)); LOG_TRACE(logger, "Inaccessible query result found for query {}", doubleQuoteString(key.query_string));
return; return;
@ -509,7 +507,7 @@ QueryCache::Writer QueryCache::createWriter(const Key & key, std::chrono::millis
/// Update the per-user cache quotas with the values stored in the query context. This happens per query which writes into the query /// Update the per-user cache quotas with the values stored in the query context. This happens per query which writes into the query
/// cache. Obviously, this is overkill but I could find the good place to hook into which is called when the settings profiles in /// cache. Obviously, this is overkill but I could find the good place to hook into which is called when the settings profiles in
/// users.xml change. /// users.xml change.
cache.setQuotaForUser(key.user_name, max_query_cache_size_in_bytes_quota, max_query_cache_entries_quota); cache.setQuotaForUser(key.user_id, max_query_cache_size_in_bytes_quota, max_query_cache_entries_quota);
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
return Writer(cache, key, max_entry_size_in_bytes, max_entry_size_in_rows, min_query_runtime, squash_partial_results, max_block_size); return Writer(cache, key, max_entry_size_in_bytes, max_entry_size_in_rows, min_query_runtime, squash_partial_results, max_block_size);

View File

@ -4,9 +4,12 @@
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Core/Block.h> #include <Core/Block.h>
#include <Parsers/IAST_fwd.h> #include <Parsers/IAST_fwd.h>
#include <Processors/Sources/SourceFromChunks.h>
#include <Processors/Chunk.h> #include <Processors/Chunk.h>
#include <Processors/Sources/SourceFromChunks.h>
#include <QueryPipeline/Pipe.h> #include <QueryPipeline/Pipe.h>
#include <base/UUID.h>
#include <optional>
namespace DB namespace DB
{ {
@ -51,14 +54,13 @@ public:
/// Result metadata for constructing the pipe. /// Result metadata for constructing the pipe.
const Block header; const Block header;
/// The name, id and current roles of the user who executed the query. /// The id and current roles of the user who executed the query.
/// These members are necessary to ensure that a (non-shared, see below) entry can only be written and read by the same user with /// These members are necessary to ensure that a (non-shared, see below) entry can only be written and read by the same user with
/// the same roles. Example attack scenarios: /// the same roles. Example attack scenarios:
/// - after DROP USER, it must not be possible to create a new user with with the dropped user name and access the dropped user's /// - after DROP USER, it must not be possible to create a new user with with the dropped user name and access the dropped user's
/// query cache entries /// query cache entries
/// - different roles of the same user may be tied to different row-level policies. It must not be possible to switch role and /// - different roles of the same user may be tied to different row-level policies. It must not be possible to switch role and
/// access another role's cache entries /// access another role's cache entries
const String user_name;
std::optional<UUID> user_id; std::optional<UUID> user_id;
std::vector<UUID> current_user_roles; std::vector<UUID> current_user_roles;
@ -82,13 +84,13 @@ public:
/// Ctor to construct a Key for writing into query cache. /// Ctor to construct a Key for writing into query cache.
Key(ASTPtr ast_, Key(ASTPtr ast_,
Block header_, Block header_,
const String & user_name_, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_,
bool is_shared_, bool is_shared_,
std::chrono::time_point<std::chrono::system_clock> expires_at_, std::chrono::time_point<std::chrono::system_clock> expires_at_,
bool is_compressed); bool is_compressed);
/// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name). /// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name).
Key(ASTPtr ast_, const String & user_name_, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_); Key(ASTPtr ast_, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_);
bool operator==(const Key & other) const; bool operator==(const Key & other) const;
}; };

View File

@ -1010,7 +1010,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
{ {
if (can_use_query_cache && settings.enable_reads_from_query_cache) if (can_use_query_cache && settings.enable_reads_from_query_cache)
{ {
QueryCache::Key key(ast, context->getUserName(), context->getUserID(), context->getCurrentRoles()); QueryCache::Key key(ast, context->getUserID(), context->getCurrentRoles());
QueryCache::Reader reader = query_cache->createReader(key); QueryCache::Reader reader = query_cache->createReader(key);
if (reader.hasCacheEntryForKey()) if (reader.hasCacheEntryForKey())
{ {
@ -1123,7 +1123,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
{ {
QueryCache::Key key( QueryCache::Key key(
ast, res.pipeline.getHeader(), ast, res.pipeline.getHeader(),
context->getUserName(), context->getUserID(), context->getCurrentRoles(), context->getUserID(), context->getCurrentRoles(),
settings.query_cache_share_between_users, settings.query_cache_share_between_users,
std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl),
settings.query_cache_compress_entries); settings.query_cache_compress_entries);

View File

@ -43,10 +43,9 @@ void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr
for (const auto & [key, query_result] : content) for (const auto & [key, query_result] : content)
{ {
/// Showing other user's queries is considered a security risk /// Showing other user's queries is considered a security risk
const bool is_same_user_name = (key.user_name == user_name);
const bool is_same_user_id = ((!key.user_id.has_value() && !user_id.has_value()) || (key.user_id.has_value() && user_id.has_value() && *key.user_id == *user_id)); const bool is_same_user_id = ((!key.user_id.has_value() && !user_id.has_value()) || (key.user_id.has_value() && user_id.has_value() && *key.user_id == *user_id));
const bool is_same_current_user_roles = (key.current_user_roles == current_user_roles); const bool is_same_current_user_roles = (key.current_user_roles == current_user_roles);
if (!key.is_shared && (!is_same_user_name || !is_same_user_id || !is_same_current_user_roles)) if (!key.is_shared && (!is_same_user_id || !is_same_current_user_roles))
continue; continue;
res_columns[0]->insert(key.query_string); /// approximates the original query string res_columns[0]->insert(key.query_string); /// approximates the original query string