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/SLRUCachePolicy.h>
#include <base/UUID.h>
#include <base/defines.h>
#include <atomic>
#include <cassert>
#include <chrono>
#include <memory>
#include <mutex>
#include <optional>
#include <unordered_map>
#include <base/defines.h>
namespace DB
{
@ -227,10 +227,10 @@ public:
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);
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;

View File

@ -2,10 +2,11 @@
#include <Common/Exception.h>
#include <Common/ICachePolicyUserQuota.h>
#include <base/UUID.h>
#include <functional>
#include <memory>
#include <mutex>
#include <optional>
namespace DB
{
@ -43,7 +44,7 @@ public:
virtual void setMaxCount(size_t /*max_count*/) = 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
/// 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
#include <base/UUID.h>
#include <base/types.h>
#include <optional>
namespace DB
{
@ -15,14 +18,14 @@ class ICachePolicyUserQuota
{
public:
/// 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.
virtual void increaseActual(const String & user_name, size_t entry_size_in_bytes) = 0;
virtual void decreaseActual(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(std::optional<UUID> user_id, size_t entry_size_in_bytes) = 0;
/// 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;
};
@ -33,10 +36,10 @@ using CachePolicyUserQuotaPtr = std::unique_ptr<ICachePolicyUserQuota>;
class NoCachePolicyUserQuota : public ICachePolicyUserQuota
{
public:
void setQuotaForUser(const String & /*user_name*/, 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 decreaseActual(const String & /*user_name*/, size_t /*entry_size_in_bytes*/) override {}
bool approveWrite(const String & /*user_name*/, size_t /*entry_size_in_bytes*/) const override { return true; }
void setQuotaForUser(std::optional<UUID> /*user_id*/, size_t /*max_size_in_bytes*/, size_t /*max_entries*/) override {}
void increaseActual(std::optional<UUID> /*user_id*/, size_t /*entry_size_in_bytes*/) override {}
void decreaseActual(std::optional<UUID> /*user_id*/, size_t /*entry_size_in_bytes*/) override {}
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
#include <Common/ICachePolicy.h>
#include <base/UUID.h>
#include <limits>
#include <optional>
#include <unordered_map>
namespace DB
@ -11,37 +13,47 @@ namespace DB
class PerUserTTLCachePolicyUserQuota : public ICachePolicyUserQuota
{
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.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);
actual[user_name].size_in_bytes -= entry_size_in_bytes;
chassert(actual.contains(*user_id));
chassert(actual[user_name].num_items >= 1);
actual[user_name].num_items -= 1;
chassert(actual[*user_id].size_in_bytes >= entry_size_in_bytes);
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
if (it_actual != actual.end())
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
if (it_quota != quotas.end())
quota_for_user = it_quota->second;
@ -69,10 +81,10 @@ public:
size_t num_items = 0;
};
/// user name --> cache size quota (in bytes) / number of items quota
std::map<String, Resources> quotas;
/// user name --> actual cache usage (in bytes) / number of items
std::map<String, Resources> actual;
/// user id --> cache size quota (in bytes) / number of items quota
std::map<UUID, Resources> quotas;
/// user id --> actual cache usage (in bytes) / number of items
std::map<UUID, Resources> actual;
};
@ -132,7 +144,7 @@ public:
if (it == cache.end())
return;
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);
size_in_bytes -= sz;
}
@ -169,7 +181,7 @@ public:
/// Checks against per-user limits
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())
@ -179,7 +191,7 @@ public:
if (is_stale_function(it->first))
{
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);
size_in_bytes -= sz;
}
@ -193,14 +205,14 @@ public:
if (auto it = cache.find(key); it != cache.end())
{
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
size_in_bytes -= sz;
}
cache[key] = std::move(mapped); // (*)
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(
ASTPtr ast_,
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_,
std::chrono::time_point<std::chrono::system_clock> expires_at_,
bool is_compressed_)
: ast(removeQueryCacheSettings(ast_))
, header(header_)
, user_name(user_name_)
, user_id(user_id_)
, current_user_roles(current_user_roles_)
, 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(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::Key(ASTPtr ast_, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_)
: 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_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_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));
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
/// 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.
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);
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 <Core/Block.h>
#include <Parsers/IAST_fwd.h>
#include <Processors/Sources/SourceFromChunks.h>
#include <Processors/Chunk.h>
#include <Processors/Sources/SourceFromChunks.h>
#include <QueryPipeline/Pipe.h>
#include <base/UUID.h>
#include <optional>
namespace DB
{
@ -51,14 +54,13 @@ public:
/// Result metadata for constructing the pipe.
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
/// 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
/// 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
/// access another role's cache entries
const String user_name;
std::optional<UUID> user_id;
std::vector<UUID> current_user_roles;
@ -82,13 +84,13 @@ public:
/// Ctor to construct a Key for writing into query cache.
Key(ASTPtr ast_,
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_,
std::chrono::time_point<std::chrono::system_clock> expires_at_,
bool is_compressed);
/// 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;
};

View File

@ -1010,7 +1010,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
{
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);
if (reader.hasCacheEntryForKey())
{
@ -1123,7 +1123,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
{
QueryCache::Key key(
ast, res.pipeline.getHeader(),
context->getUserName(), context->getUserID(), context->getCurrentRoles(),
context->getUserID(), context->getCurrentRoles(),
settings.query_cache_share_between_users,
std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl),
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)
{
/// 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_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;
res_columns[0]->insert(key.query_string); /// approximates the original query string