mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Apply settings for explain subquery as well.
This commit is contained in:
parent
f6ac225f98
commit
d2c28fb555
@ -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();
|
||||
|
@ -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);
|
||||
|
Loading…
Reference in New Issue
Block a user