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)
|
if (!async_insert)
|
||||||
{
|
{
|
||||||
/// We need to start the (implicit) transaction before getting the interpreter as this will get links to the latest snapshots
|
/// 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();
|
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
|
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
|
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())
|
&& res.pipeline.pulling())
|
||||||
{
|
{
|
||||||
QueryCache::Key key(
|
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.
|
/// then add a processor on top of the pipeline which stores the result in the query cache.
|
||||||
if (!read_result_from_query_cache
|
if (!read_result_from_query_cache
|
||||||
&& query_cache != nullptr
|
&& 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()
|
&& res.pipeline.pulling()
|
||||||
&& (!astContainsNonDeterministicFunctions(ast, context) || settings.query_cache_store_results_of_queries_with_nondeterministic_functions))
|
&& (!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,
|
auto finish_callback = [elem,
|
||||||
context,
|
context,
|
||||||
ast,
|
ast,
|
||||||
allow_experimental_query_cache = settings.allow_experimental_query_cache,
|
can_use_query_cache = can_use_query_cache,
|
||||||
use_query_cache = settings.use_query_cache,
|
|
||||||
enable_writes_to_query_cache = settings.enable_writes_to_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,
|
query_cache_store_results_of_queries_with_nondeterministic_functions = settings.query_cache_store_results_of_queries_with_nondeterministic_functions,
|
||||||
log_queries,
|
log_queries,
|
||||||
@ -913,7 +916,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
|
|||||||
auto query_cache = context->getQueryCache();
|
auto query_cache = context->getQueryCache();
|
||||||
if (query_cache != nullptr
|
if (query_cache != nullptr
|
||||||
&& pulling_pipeline
|
&& 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))
|
&& (!astContainsNonDeterministicFunctions(ast, context) || query_cache_store_results_of_queries_with_nondeterministic_functions))
|
||||||
{
|
{
|
||||||
query_pipeline.finalizeWriteInQueryCache();
|
query_pipeline.finalizeWriteInQueryCache();
|
||||||
|
@ -11,6 +11,8 @@
|
|||||||
#include <TableFunctions/registerTableFunctions.h>
|
#include <TableFunctions/registerTableFunctions.h>
|
||||||
#include <Processors/Executors/PullingPipelineExecutor.h>
|
#include <Processors/Executors/PullingPipelineExecutor.h>
|
||||||
#include <Analyzer/TableFunctionNode.h>
|
#include <Analyzer/TableFunctionNode.h>
|
||||||
|
#include <Interpreters/InterpreterSetQuery.h>
|
||||||
|
#include <Interpreters/Context.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
@ -123,7 +125,10 @@ static Block executeMonoBlock(QueryPipeline & pipeline)
|
|||||||
StoragePtr TableFunctionExplain::executeImpl(
|
StoragePtr TableFunctionExplain::executeImpl(
|
||||||
const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
|
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);
|
Block block = executeMonoBlock(blockio.pipeline);
|
||||||
|
|
||||||
StorageID storage_id(getDatabaseName(), table_name);
|
StorageID storage_id(getDatabaseName(), table_name);
|
||||||
|
Loading…
Reference in New Issue
Block a user