diff --git a/src/Common/CacheBase.h b/src/Common/CacheBase.h index 1cbfcc2165a5..a809136f4511 100644 --- a/src/Common/CacheBase.h +++ b/src/Common/CacheBase.h @@ -5,15 +5,15 @@ #include #include +#include +#include + #include -#include -#include #include #include +#include #include -#include - namespace DB { @@ -227,10 +227,10 @@ class CacheBase 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(const 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; diff --git a/src/Common/ICachePolicy.h b/src/Common/ICachePolicy.h index 189af4db19be..8aa75d1d81fa 100644 --- a/src/Common/ICachePolicy.h +++ b/src/Common/ICachePolicy.h @@ -2,10 +2,11 @@ #include #include +#include #include #include -#include +#include namespace DB { @@ -43,7 +44,7 @@ class ICachePolicy 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(const 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. diff --git a/src/Common/ICachePolicyUserQuota.h b/src/Common/ICachePolicyUserQuota.h index 717cb916f856..6fa4f7947cb1 100644 --- a/src/Common/ICachePolicyUserQuota.h +++ b/src/Common/ICachePolicyUserQuota.h @@ -1,5 +1,6 @@ #pragma once +#include #include namespace DB @@ -15,14 +16,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(const 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(const UUID & user_id, size_t entry_size_in_bytes) = 0; + virtual void decreaseActual(const 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(const UUID & user_id, size_t entry_size_in_bytes) const = 0; virtual ~ICachePolicyUserQuota() = default; }; @@ -33,10 +34,10 @@ using CachePolicyUserQuotaPtr = std::unique_ptr; 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(const UUID & /*user_id*/, size_t /*max_size_in_bytes*/, size_t /*max_entries*/) override {} + void increaseActual(const UUID & /*user_id*/, size_t /*entry_size_in_bytes*/) override {} + void decreaseActual(const UUID & /*user_id*/, size_t /*entry_size_in_bytes*/) override {} + bool approveWrite(const UUID & /*user_id*/, size_t /*entry_size_in_bytes*/) const override { return true; } }; diff --git a/src/Common/TTLCachePolicy.h b/src/Common/TTLCachePolicy.h index 98708c653c3d..e7880df0dfc9 100644 --- a/src/Common/TTLCachePolicy.h +++ b/src/Common/TTLCachePolicy.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -11,68 +12,63 @@ 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(const UUID & user_id, size_t max_size_in_bytes, size_t max_entries) override { - quotas[user_name] = {max_size_in_bytes, max_entries}; + quotas[user_id] = {max_size_in_bytes, max_entries}; } - void increaseActual(const String & user_name, size_t entry_size_in_bytes) override + void increaseActual(const UUID & user_id, size_t entry_size_in_bytes) override { - auto & actual_for_user = actual[user_name]; + 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(const UUID & user_id, size_t entry_size_in_bytes) override { - chassert(actual.contains(user_name)); + chassert(actual.contains(user_id)); - chassert(actual[user_name].size_in_bytes >= entry_size_in_bytes); - actual[user_name].size_in_bytes -= entry_size_in_bytes; + chassert(actual[user_id].size_in_bytes >= entry_size_in_bytes); + actual[user_id].size_in_bytes -= entry_size_in_bytes; - chassert(actual[user_name].num_items >= 1); - actual[user_name].num_items -= 1; + 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(const UUID & user_id, size_t entry_size_in_bytes) const override { - auto it_actual = actual.find(user_name); + 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::max(), .num_items = std::numeric_limits::max()}; /// assume no threshold if no quota is found if (it_quota != quotas.end()) quota_for_user = it_quota->second; - /// Special case: A quota configured as 0 means no threshold if (quota_for_user.size_in_bytes == 0) quota_for_user.size_in_bytes = std::numeric_limits::max(); if (quota_for_user.num_items == 0) quota_for_user.num_items = std::numeric_limits::max(); - /// Check size quota if (actual_for_user.size_in_bytes + entry_size_in_bytes >= quota_for_user.size_in_bytes) return false; - /// Check items quota if (quota_for_user.num_items + 1 >= quota_for_user.num_items) return false; - return true; } - struct Resources { size_t size_in_bytes = 0; size_t num_items = 0; }; - /// user name --> cache size quota (in bytes) / number of items quota - std::map quotas; - /// user name --> actual cache usage (in bytes) / number of items - std::map actual; + /// user id --> cache size quota (in bytes) / number of items quota + std::map quotas; + /// user id --> actual cache usage (in bytes) / number of items + std::map actual; }; @@ -132,7 +128,8 @@ class TTLCachePolicy : public ICachePolicysecond); - Base::user_quotas->decreaseActual(it->first.user_name, sz); + if (it->first.user_id.has_value()) + Base::user_quotas->decreaseActual(*it->first.user_id, sz); cache.erase(it); size_in_bytes -= sz; } @@ -169,7 +166,9 @@ class TTLCachePolicy : public ICachePolicyapproveWrite(key.user_name, entry_size_in_bytes); + if (key.user_id.has_value()) + return Base::user_quotas->approveWrite(*key.user_id, entry_size_in_bytes); + return true; }; if (!sufficient_space_in_cache() || !sufficient_space_in_cache_for_user()) @@ -179,7 +178,8 @@ class TTLCachePolicy : public ICachePolicyfirst)) { size_t sz = weight_function(*it->second); - Base::user_quotas->decreaseActual(it->first.user_name, sz); + if (it->first.user_id.has_value()) + Base::user_quotas->decreaseActual(*it->first.user_id, sz); it = cache.erase(it); size_in_bytes -= sz; } @@ -193,14 +193,16 @@ class TTLCachePolicy : public ICachePolicysecond); - Base::user_quotas->decreaseActual(it->first.user_name, sz); + if (it->first.user_id.has_value()) + 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); + if (key.user_id.has_value()) + Base::user_quotas->increaseActual(*key.user_id, entry_size_in_bytes); } } diff --git a/src/Interpreters/Cache/QueryCache.cpp b/src/Interpreters/Cache/QueryCache.cpp index 5283e307f2d8..b6ab85b36642 100644 --- a/src/Interpreters/Cache/QueryCache.cpp +++ b/src/Interpreters/Cache/QueryCache.cpp @@ -128,12 +128,14 @@ String queryStringFromAST(ASTPtr ast) QueryCache::Key::Key( ASTPtr ast_, Block header_, - const String & user_name_, bool is_shared_, + std::optional user_id_, const std::vector & current_user_roles_, + bool is_shared_, std::chrono::time_point 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_) , expires_at(expires_at_) , is_compressed(is_compressed_) @@ -141,8 +143,8 @@ QueryCache::Key::Key( { } -QueryCache::Key::Key(ASTPtr ast_, const String & user_name_) - : QueryCache::Key(ast_, {}, user_name_, 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 user_id_, const std::vector & 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 { } @@ -395,7 +397,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; - if (!entry_key.is_shared && 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_id || !is_same_current_user_roles)) { LOG_TRACE(&Poco::Logger::get("QueryCache"), "Inaccessible entry found for query {}", key.query_string); return; @@ -497,7 +501,9 @@ 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); + /// user_id == std::nullopt is the internal user for which no quota can be configured + if (key.user_id.has_value()) + 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); diff --git a/src/Interpreters/Cache/QueryCache.h b/src/Interpreters/Cache/QueryCache.h index 44099cfe61d3..2298162a761d 100644 --- a/src/Interpreters/Cache/QueryCache.h +++ b/src/Interpreters/Cache/QueryCache.h @@ -1,11 +1,15 @@ #pragma once #include +#include #include #include -#include #include +#include #include +#include + +#include namespace DB { @@ -50,8 +54,15 @@ class QueryCache /// Result metadata for constructing the pipe. const Block header; - /// The user who executed the query. - const String user_name; + /// 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 + std::optional user_id; + std::vector current_user_roles; /// If the associated entry can be read by other users. In general, sharing is a bad idea: First, it is unlikely that different /// users pose the same queries. Second, sharing potentially breaches security. E.g. User A should not be able to bypass row @@ -73,12 +84,13 @@ class QueryCache /// Ctor to construct a Key for writing into query cache. Key(ASTPtr ast_, Block header_, - const String & user_name_, bool is_shared_, + std::optional user_id_, const std::vector & current_user_roles_, + bool is_shared_, std::chrono::time_point 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_); + Key(ASTPtr ast_, std::optional user_id_, const std::vector & current_user_roles_); bool operator==(const Key & other) const; }; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 29b8035a490c..475aabd2ff1b 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1169,6 +1169,16 @@ boost::container::flat_set Context::getCurrentRoles() const return getRolesInfo()->current_roles; } +std::vector Context::getCurrentRolesAsStdVector() const +{ + const auto & roles = getRolesInfo()->current_roles; + std::vector res; + res.reserve(roles.size()); + for (auto uuid : roles) + res.push_back(uuid); + return res; +} + boost::container::flat_set Context::getEnabledRoles() const { return getRolesInfo()->enabled_roles; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 59c1ac19c8ac..dd0974aeabbb 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -549,6 +549,7 @@ class Context: public std::enable_shared_from_this void setCurrentRoles(const std::vector & current_roles_); void setCurrentRolesDefault(); boost::container::flat_set getCurrentRoles() const; + std::vector getCurrentRolesAsStdVector() const; boost::container::flat_set getEnabledRoles() const; std::shared_ptr getRolesInfo() const; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 8c87c5883eb7..a8232e990e66 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -990,7 +990,7 @@ static std::tuple executeQueryImpl( { if (can_use_query_cache && settings.enable_reads_from_query_cache) { - QueryCache::Key key(ast, context->getUserName()); + QueryCache::Key key(ast, context->getUserID(), context->getCurrentRolesAsStdVector()); QueryCache::Reader reader = query_cache->createReader(key); if (reader.hasCacheEntryForKey()) { @@ -1091,7 +1091,8 @@ static std::tuple executeQueryImpl( { QueryCache::Key key( ast, res.pipeline.getHeader(), - context->getUserName(), settings.query_cache_share_between_users, + context->getUserID(), context->getCurrentRolesAsStdVector(), + settings.query_cache_share_between_users, std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl), settings.query_cache_compress_entries); diff --git a/src/Storages/System/StorageSystemQueryCache.cpp b/src/Storages/System/StorageSystemQueryCache.cpp index 03757101ddfe..5146d8fd906a 100644 --- a/src/Storages/System/StorageSystemQueryCache.cpp +++ b/src/Storages/System/StorageSystemQueryCache.cpp @@ -30,18 +30,20 @@ StorageSystemQueryCache::StorageSystemQueryCache(const StorageID & table_id_) void StorageSystemQueryCache::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { QueryCachePtr query_cache = context->getQueryCache(); - if (!query_cache) return; - std::vector content = query_cache->dump(); const String & user_name = context->getUserName(); + std::optional user_id = context->getUserID(); + std::vector current_user_roles = context->getCurrentRolesAsStdVector(); for (const auto & [key, query_result] : content) { /// Showing other user's queries is considered a security risk - if (!key.is_shared && 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_id || !is_same_current_user_roles)) continue; res_columns[0]->insert(key.query_string); /// approximates the original query string diff --git a/tests/queries/0_stateless/02494_query_cache_user_isolation.reference b/tests/queries/0_stateless/02494_query_cache_user_isolation.reference new file mode 100644 index 000000000000..f8c4b31b22ae --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_user_isolation.reference @@ -0,0 +1,28 @@ +Attack 1 +0 +system.query_cache with old user 1 +0 +0 1 +1 0 +system.query_cache with new user 0 +0 +0 1 +1 0 +0 1 +Attack 2 +-- policy_1 test +1 1 +3 1 +6 1 +-- policy_2 test +2 2 +5 2 +8 2 +-- policy_1 with query cache test +1 1 +3 1 +6 1 +-- policy_2 with query cache test +2 2 +5 2 +8 2 diff --git a/tests/queries/0_stateless/02494_query_cache_user_isolation.sh b/tests/queries/0_stateless/02494_query_cache_user_isolation.sh new file mode 100644 index 000000000000..d55e2460619a --- /dev/null +++ b/tests/queries/0_stateless/02494_query_cache_user_isolation.sh @@ -0,0 +1,110 @@ +#!/usr/bin/env bash +# Tags: no-parallel, no-fasttest, long +# Tag no-parallel: Messes with internal cache +# no-fasttest: Produces wrong results in fasttest, unclear why, didn't reproduce locally. +# long: Sloooow ... + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# -- Attack 1: +# - create a user, +# - run a query whose result is stored in the query cache, +# - drop the user, recreate it with the same name +# - test that the cache entry is inaccessible + +echo "Attack 1" + +rnd=`tr -dc 1-9