Include settings into query cache key

This commit is contained in:
Robert Schulze 2024-05-21 17:29:00 +00:00
parent 0bfa56b468
commit dac31fb92a
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
7 changed files with 113 additions and 41 deletions

View File

@ -126,6 +126,11 @@ bool astContainsSystemTables(ASTPtr ast, ContextPtr context)
namespace
{
bool isQueryCacheRelatedSetting(const String & setting_name)
{
return setting_name.starts_with("query_cache_") || setting_name.ends_with("_query_cache");
}
class RemoveQueryCacheSettingsMatcher
{
public:
@ -141,7 +146,7 @@ public:
auto is_query_cache_related_setting = [](const auto & change)
{
return change.name.starts_with("query_cache_") || change.name.ends_with("_query_cache");
return isQueryCacheRelatedSetting(change.name);
};
std::erase_if(set_clause->changes, is_query_cache_related_setting);
@ -177,11 +182,11 @@ ASTPtr removeQueryCacheSettings(ASTPtr ast)
return transformed_ast;
}
IAST::Hash calculateAstHash(ASTPtr ast, const String & current_database)
IAST::Hash calculateAstHash(ASTPtr ast, const String & current_database, const Settings & settings)
{
ast = removeQueryCacheSettings(ast);
/// Hash the AST, it must consider aliases (issue #56258)
/// Hash the AST, we must consider aliases (issue #56258)
SipHash hash;
ast->updateTreeHash(hash, /*ignore_aliases=*/ false);
@ -189,6 +194,25 @@ IAST::Hash calculateAstHash(ASTPtr ast, const String & current_database)
/// tables (issue #64136)
hash.update(current_database);
/// Finally, hash the (changed) settings as they might affect the query result (e.g. think of settings `additional_table_filters` and `limit`).
/// Note: allChanged() returns the settings in random order. Also, update()-s of the composite hash must be done in deterministic order.
/// Therefore, collect and sort the settings first, then hash them.
Settings::Range changed_settings = settings.allChanged();
std::vector<std::pair<String, String>> changed_settings_sorted; /// (name, value)
for (const auto & setting : changed_settings)
{
const String & name = setting.getName();
const String & value = setting.getValueString();
if (!isQueryCacheRelatedSetting(name)) /// see removeQueryCacheSettings() why this is a good idea
changed_settings_sorted.push_back({name, value});
}
std::sort(changed_settings_sorted.begin(), changed_settings_sorted.end(), [](auto & lhs, auto & rhs) { return lhs.first < rhs.first; });
for (const auto & setting : changed_settings_sorted)
{
hash.update(setting.first);
hash.update(setting.second);
}
return getSipHash128AsPair(hash);
}
@ -204,12 +228,13 @@ String queryStringFromAST(ASTPtr ast)
QueryCache::Key::Key(
ASTPtr ast_,
const String & current_database,
const Settings & settings,
Block header_,
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_hash(calculateAstHash(ast_, current_database))
: ast_hash(calculateAstHash(ast_, current_database, settings))
, header(header_)
, user_id(user_id_)
, current_user_roles(current_user_roles_)
@ -220,8 +245,8 @@ QueryCache::Key::Key(
{
}
QueryCache::Key::Key(ASTPtr ast_, const String & current_database, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_)
: QueryCache::Key(ast_, current_database, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST, current database, user name/roles
QueryCache::Key::Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_)
: QueryCache::Key(ast_, current_database, settings, {}, user_id_, current_user_roles_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST, current database, user name/roles
{
}

View File

@ -14,6 +14,8 @@
namespace DB
{
struct Settings;
/// Does AST contain non-deterministic functions like rand() and now()?
bool astContainsNonDeterministicFunctions(ASTPtr ast, ContextPtr context);
@ -89,6 +91,7 @@ public:
/// Ctor to construct a Key for writing into query cache.
Key(ASTPtr ast_,
const String & current_database,
const Settings & settings,
Block header_,
std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_,
bool is_shared_,
@ -96,7 +99,7 @@ public:
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 & current_database, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_);
Key(ASTPtr ast_, const String & current_database, const Settings & settings, std::optional<UUID> user_id_, const std::vector<UUID> & current_user_roles_);
bool operator==(const Key & other) const;
};

View File

@ -1101,7 +1101,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
{
if (can_use_query_cache && settings.enable_reads_from_query_cache)
{
QueryCache::Key key(ast, context->getCurrentDatabase(), context->getUserID(), context->getCurrentRoles());
QueryCache::Key key(ast, context->getCurrentDatabase(), settings, context->getUserID(), context->getCurrentRoles());
QueryCache::Reader reader = query_cache->createReader(key);
if (reader.hasCacheEntryForKey())
{
@ -1224,7 +1224,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
&& (!ast_contains_system_tables || system_table_handling == QueryCacheSystemTableHandling::Save))
{
QueryCache::Key key(
ast, context->getCurrentDatabase(), res.pipeline.getHeader(),
ast, context->getCurrentDatabase(), settings, res.pipeline.getHeader(),
context->getUserID(), context->getCurrentRoles(),
settings.query_cache_share_between_users,
std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl),

View File

@ -0,0 +1,6 @@
Test (1)
1
2
Test (2)
4
4

View File

@ -0,0 +1,70 @@
-- Tags: no-parallel
-- Tag no-parallel: Messes with internal cache
-- Tests that the key of the query cache is not only formed by the query AST but also by
-- (1) the current database (`USE db`, issue #64136),
-- (2) the query settings
SELECT 'Test (1)';
SYSTEM DROP QUERY CACHE;
DROP DATABASE IF EXISTS db1;
DROP DATABASE IF EXISTS db2;
CREATE DATABASE db1;
CREATE DATABASE db2;
CREATE TABLE db1.tab(a UInt64, PRIMARY KEY a);
CREATE TABLE db2.tab(a UInt64, PRIMARY KEY a);
INSERT INTO db1.tab values(1);
INSERT INTO db2.tab values(2);
USE db1;
SELECT * FROM tab SETTINGS use_query_cache=1;
USE db2;
SELECT * FROM tab SETTINGS use_query_cache=1;
DROP DATABASE db1;
DROP DATABASE db2;
SYSTEM DROP QUERY CACHE;
SELECT 'Test (2)';
-- test with query-level settings
SELECT 1 SETTINGS use_query_cache = 1, limit = 1, use_skip_indexes = 0 Format Null;
SELECT 1 SETTINGS use_query_cache = 1, use_skip_indexes = 0 Format Null;
SELECT 1 SETTINGS use_query_cache = 1, use_skip_indexes = 1 Format Null;
SELECT 1 SETTINGS use_query_cache = 1, max_block_size = 1 Format Null;
-- 4x the same query but with different settings each. There should yield four entries in the query cache.
SELECT count(query) FROM system.query_cache;
SYSTEM DROP QUERY CACHE;
-- test with mixed session-level/query-level settings
SET use_query_cache = 1;
SET limit = 1;
SELECT 1 SETTINGS use_skip_indexes = 0 Format Null;
SET limit = default;
SET use_skip_indexes = 0;
SELECT 1 Format Null;
SET use_skip_indexes = 1;
SELECT 1 SETTINGS use_skip_indexes = 1 Format Null;
SET use_skip_indexes = default;
SET max_block_size = 1;
SELECT 1 Format Null;
SET max_block_size = default;
SET use_query_cache = default;
-- 4x the same query but with different settings each. There should yield four entries in the query cache.
SELECT count(query) FROM system.query_cache;
SYSTEM DROP QUERY CACHE;

View File

@ -1,30 +0,0 @@
-- Tags: no-parallel, no-fasttest
-- Tag no-fasttest: Depends on OpenSSL
-- Tag no-parallel: Messes with internal cache
-- Test for issue #64136
SYSTEM DROP QUERY CACHE;
DROP DATABASE IF EXISTS db1;
DROP DATABASE IF EXISTS db2;
CREATE DATABASE db1;
CREATE DATABASE db2;
CREATE TABLE db1.tab(a UInt64, PRIMARY KEY a);
CREATE TABLE db2.tab(a UInt64, PRIMARY KEY a);
INSERT INTO db1.tab values(1);
INSERT INTO db2.tab values(2);
USE db1;
SELECT * FROM tab SETTINGS use_query_cache=1;
USE db2;
SELECT * FROM tab SETTINGS use_query_cache=1;
DROP DATABASE db1;
DROP DATABASE db2;
SYSTEM DROP QUERY CACHE;