mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-02 12:32:04 +00:00
Get rid of QueryCache::user_name
This commit is contained in:
parent
bca0f144e2
commit
e9b6f413b8
@ -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;
|
||||||
|
@ -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.
|
||||||
|
@ -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; }
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -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);
|
||||||
|
@ -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;
|
||||||
};
|
};
|
||||||
|
@ -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);
|
||||||
|
@ -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
|
||||||
|
Loading…
Reference in New Issue
Block a user