Apply settings for explain subquery as well.

This commit is contained in:
Nikolai Kochetov 2023-03-10 18:29:13 +00:00
parent f6ac225f98
commit d2c28fb555
2 changed files with 14 additions and 6 deletions

View File

@ -620,6 +620,10 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
}
}
bool can_use_query_cache =
settings.allow_experimental_query_cache && settings.use_query_cache
&& !ast->as<ASTExplainQuery>();
if (!async_insert)
{
/// We need to start the (implicit) transaction before getting the interpreter as this will get links to the latest snapshots
@ -706,7 +710,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
auto query_cache = context->getQueryCache();
bool read_result_from_query_cache = false; /// a query must not read from *and* write to the query cache at the same time
if (query_cache != nullptr
&& (settings.allow_experimental_query_cache && settings.use_query_cache && settings.enable_reads_from_query_cache)
&& (can_use_query_cache && settings.enable_reads_from_query_cache)
&& res.pipeline.pulling())
{
QueryCache::Key key(
@ -727,7 +731,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
/// then add a processor on top of the pipeline which stores the result in the query cache.
if (!read_result_from_query_cache
&& query_cache != nullptr
&& settings.allow_experimental_query_cache && settings.use_query_cache && settings.enable_writes_to_query_cache
&& can_use_query_cache && settings.enable_writes_to_query_cache
&& res.pipeline.pulling()
&& (!astContainsNonDeterministicFunctions(ast, context) || settings.query_cache_store_results_of_queries_with_nondeterministic_functions))
{
@ -895,8 +899,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
auto finish_callback = [elem,
context,
ast,
allow_experimental_query_cache = settings.allow_experimental_query_cache,
use_query_cache = settings.use_query_cache,
can_use_query_cache = can_use_query_cache,
enable_writes_to_query_cache = settings.enable_writes_to_query_cache,
query_cache_store_results_of_queries_with_nondeterministic_functions = settings.query_cache_store_results_of_queries_with_nondeterministic_functions,
log_queries,
@ -913,7 +916,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
auto query_cache = context->getQueryCache();
if (query_cache != nullptr
&& pulling_pipeline
&& allow_experimental_query_cache && use_query_cache && enable_writes_to_query_cache
&& can_use_query_cache && enable_writes_to_query_cache
&& (!astContainsNonDeterministicFunctions(ast, context) || query_cache_store_results_of_queries_with_nondeterministic_functions))
{
query_pipeline.finalizeWriteInQueryCache();

View File

@ -11,6 +11,8 @@
#include <TableFunctions/registerTableFunctions.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Analyzer/TableFunctionNode.h>
#include <Interpreters/InterpreterSetQuery.h>
#include <Interpreters/Context.h>
namespace DB
{
@ -123,7 +125,10 @@ static Block executeMonoBlock(QueryPipeline & pipeline)
StoragePtr TableFunctionExplain::executeImpl(
const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
{
BlockIO blockio = getInterpreter(context).execute();
/// To support settings inside explain subquery.
auto mutable_context = Context::createCopy(context);
InterpreterSetQuery::applySettingsFromQuery(query, mutable_context);
BlockIO blockio = getInterpreter(mutable_context).execute();
Block block = executeMonoBlock(blockio.pipeline);
StorageID storage_id(getDatabaseName(), table_name);