Normalize AST before QRC insert/lookup

This commit is contained in:
Robert Schulze 2022-12-16 15:38:53 +00:00
parent 021311be8d
commit 20a7af926e
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
6 changed files with 140 additions and 9 deletions

View File

@ -2,7 +2,9 @@
#include <Functions/FunctionFactory.h>
#include <Interpreters/Context.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSetQuery.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
#include <Common/logger_useful.h>
#include <Common/ProfileEvents.h>
@ -36,6 +38,77 @@ bool astContainsNonDeterministicFunctions(ASTPtr ast, ContextPtr context)
return has_non_cacheable_functions;
}
namespace
{
class RemoveQueryResultCacheSettingsMatcher
{
public:
struct Data {};
static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; }
static void visit(ASTPtr & ast, Data & data)
{
if (auto * func = ast->as<ASTSetQuery>())
visit(*func, ast, data);
}
static void visit(ASTSetQuery & func, ASTPtr &, const Data &)
{
assert(!func.is_standalone);
auto is_query_result_cache_related_setting = [](const auto & change)
{
return change.name.starts_with("enable_experimental_query_result_cache")
|| change.name.starts_with("query_result_cache");
};
std::erase_if(func.changes, is_query_result_cache_related_setting);
}
/// TODO further improve AST cleanup, e.g. remove SETTINGS clause completely if it is empty
/// E.g. SELECT 1 SETTINGS enable_experimental_query_result_cache = true
/// and SELECT 1;
/// currently don't match.
};
using RemoveQueryResultCacheSettingsVisitor = InDepthNodeVisitor<RemoveQueryResultCacheSettingsMatcher, true>;
/// Consider
/// (1) SET enable_experimental_query_result_cache = true;
/// SELECT expensiveComputation(...) SETTINGS max_threads = 64, query_result_cache_keep_seconds_alive = 300;
/// SET enable_experimental_query_result_cache = false;
/// and
/// (2) SELECT expensiveComputation(...) SETTINGS max_threads = 64, enable_experimental_query_result_cache_passive_usage = true;
/// The SELECT query in (1) and (2) is basically the same and the user expects to find the result in the query result cache. However, query
/// results are indexed by the AST of their SELECT queries and no result will be found. Insert and retrieval behave overall more natural if
/// settings related to the query result cache are erased from the AST key. Note that at this point the settings themselves have been parsed
/// already, they are not lost or discarded.
ASTPtr removeQueryResultCacheSettings(ASTPtr ast)
{
ASTPtr transformed_ast = ast->clone();
RemoveQueryResultCacheSettingsMatcher::Data visitor_data{};
RemoveQueryResultCacheSettingsVisitor visitor(visitor_data);
visitor.visit(transformed_ast);
return transformed_ast;
}
}
QueryResultCache::Key::Key(
ASTPtr ast_, String username_, String partition_key_,
Block header_, std::chrono::time_point<std::chrono::system_clock> expires_at_)
: ast(removeQueryResultCacheSettings(ast_))
, username(username_)
, partition_key(partition_key_)
, header(header_)
, expires_at(expires_at_)
{
}
bool QueryResultCache::Key::operator==(const Key & other) const
{
return ast->getTreeHash() == other.ast->getTreeHash()

View File

@ -50,6 +50,8 @@ public:
/// When does the entry expire?
const std::chrono::time_point<std::chrono::system_clock> expires_at;
Key(ASTPtr ast_, String username_, String partition_key_, Block header_, std::chrono::time_point<std::chrono::system_clock> expires_at_);
bool operator==(const Key & other) const;
String queryStringFromAst() const;
};

View File

@ -692,9 +692,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
if ((settings.enable_experimental_query_result_cache || settings.enable_experimental_query_result_cache_passive_usage)
&& query_result_cache != nullptr && res.pipeline.pulling())
{
QueryResultCache::Key key{
QueryResultCache::Key key(
ast, context->getUserName(), settings.query_result_cache_partition_key, res.pipeline.getHeader(),
std::chrono::system_clock::now() + std::chrono::seconds(settings.query_result_cache_keep_seconds_alive)};
std::chrono::system_clock::now() + std::chrono::seconds(settings.query_result_cache_keep_seconds_alive));
QueryResultCache::Reader reader = query_result_cache->createReader(key);
if (reader.hasCacheEntryForKey())
res.pipeline = QueryPipeline(reader.getPipe());
@ -703,9 +703,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
if (settings.enable_experimental_query_result_cache && query_result_cache != nullptr && res.pipeline.pulling()
&& (settings.query_result_cache_store_results_of_queries_with_nondeterministic_functions || !astContainsNonDeterministicFunctions(ast, context)))
{
QueryResultCache::Key key{
QueryResultCache::Key key(
ast, context->getUserName(), settings.query_result_cache_partition_key, res.pipeline.getHeader(),
std::chrono::system_clock::now() + std::chrono::seconds(settings.query_result_cache_keep_seconds_alive)};
std::chrono::system_clock::now() + std::chrono::seconds(settings.query_result_cache_keep_seconds_alive));
const size_t num_query_runs = query_result_cache->recordQueryRun(key);
if (num_query_runs > settings.query_result_cache_min_query_runs)

View File

@ -2,16 +2,17 @@
SYSTEM DROP QUERY RESULT CACHE;
-- this does create an entry in the query result cache
-- this does not create a cache entry ...
SET query_result_cache_max_entry_size = 0;
SELECT 1 SETTINGS enable_experimental_query_result_cache = true;
SELECT COUNT(*) FROM system.queryresult_cache;
SELECT count(*) FROM system.queryresult_cache;
SYSTEM DROP QUERY RESULT CACHE;
-- ... but this not because the query executes much faster than the milliseconds threshold
SET query_result_cache_min_query_duration = 10000;
-- ... but this does
SET query_result_cache_max_entry_size = 9999999;
SELECT 1 SETTINGS enable_experimental_query_result_cache = true;
SELECT COUNT(*) FROM system.queryresult_cache;
SELECT count(*) FROM system.queryresult_cache;
SYSTEM DROP QUERY RESULT CACHE;

View File

@ -0,0 +1,26 @@
-- { echoOn }
SYSTEM DROP QUERY RESULT CACHE;
DROP TABLE IF EXISTS old;
-- save current event counts for query result cache
CREATE TABLE old (event String, value UInt64) ENGINE=MergeTree ORDER BY event;
INSERT INTO old SELECT event, value FROM system.events WHERE event LIKE 'QueryResultCache%';
-- Run query whose result gets cached in the query result cache (QRC).
-- Besides "enable_experimental_query_result_cache", pass two knobs. We just care *that* they are passed and not about their effect.
-- One QRC-specific knob and one non-QRC-specific knob
SELECT 1 SETTINGS enable_experimental_query_result_cache = true, query_result_cache_store_results_of_queries_with_nondeterministic_functions = true, max_threads = 16;
1
SELECT COUNT(*) FROM system.queryresult_cache;
1
-- Run same query again. We want its result to be served from the QRC.
-- Technically, both SELECT 1 queries have different ASTs.
SELECT 1 SETTINGS enable_experimental_query_result_cache_passive_usage = true, max_threads = 16;
1
-- Different ASTs lead to different QRC keys. Internal normalization makes sure that the keys match regardless.
-- Verify by checking that we had a cache hit.
SELECT value = (SELECT value FROM old WHERE event = 'QueryResultCacheHits') + 1
FROM system.events
WHERE event = 'QueryResultCacheHits';
1
SYSTEM DROP QUERY RESULT CACHE;
DROP TABLE old;

View File

@ -0,0 +1,29 @@
-- { echoOn }
SYSTEM DROP QUERY RESULT CACHE;
DROP TABLE IF EXISTS old;
-- save current event counts for query result cache
CREATE TABLE old (event String, value UInt64) ENGINE=MergeTree ORDER BY event;
INSERT INTO old SELECT event, value FROM system.events WHERE event LIKE 'QueryResultCache%';
-- Run query whose result gets cached in the query result cache (QRC).
-- Besides "enable_experimental_query_result_cache", pass two knobs. We just care *that* they are passed and not about their effect.
-- One QRC-specific knob and one non-QRC-specific knob
SELECT 1 SETTINGS enable_experimental_query_result_cache = true, query_result_cache_store_results_of_queries_with_nondeterministic_functions = true, max_threads = 16;
SELECT COUNT(*) FROM system.queryresult_cache;
-- Run same query again. We want its result to be served from the QRC.
-- Technically, both SELECT 1 queries have different ASTs.
SELECT 1 SETTINGS enable_experimental_query_result_cache_passive_usage = true, max_threads = 16;
-- Different ASTs lead to different QRC keys. Internal normalization makes sure that the keys match regardless.
-- Verify by checking that we had a cache hit.
SELECT value = (SELECT value FROM old WHERE event = 'QueryResultCacheHits') + 1
FROM system.events
WHERE event = 'QueryResultCacheHits';
SYSTEM DROP QUERY RESULT CACHE;
DROP TABLE old;
-- { echoOff }