mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 00:52:02 +00:00
Merge pull request #40334 from ClickHouse/vdimir/analyze-stuck-limit
Limit number of analyze for one query, att. 2
This commit is contained in:
commit
f2cf7d7762
@ -346,7 +346,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
|
||||
M(UInt64, max_temporary_non_const_columns, 0, "Similar to the 'max_temporary_columns' setting but applies only to non-constant columns. This makes sense, because constant columns are cheap and it is reasonable to allow more of them.", 0) \
|
||||
\
|
||||
M(UInt64, max_subquery_depth, 100, "If a query has more than specified number of nested subqueries, throw an exception. This allows you to have a sanity check to protect the users of your cluster from going insane with their queries.", 0) \
|
||||
M(UInt64, max_pipeline_depth, 1000, "If a query has more than specified stages in the query pipeline, throw an exception. Pipeline has stages for every relational operator. This allows to limit the complexity of the queries.", 0) \
|
||||
M(UInt64, max_analyze_depth, 5000, "Maximum number of analyses performed by interpreter.", 0) \
|
||||
M(UInt64, max_ast_depth, 1000, "Maximum depth of query syntax tree. Checked after parsing.", 0) \
|
||||
M(UInt64, max_ast_elements, 50000, "Maximum size of query syntax tree in number of nodes. Checked after parsing.", 0) \
|
||||
M(UInt64, max_expanded_ast_elements, 500000, "Maximum size of query syntax tree in number of nodes after expansion of aliases and the asterisk.", 0) \
|
||||
@ -674,6 +674,8 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
|
||||
MAKE_OBSOLETE(M, UInt64, background_message_broker_schedule_pool_size, 16) \
|
||||
MAKE_OBSOLETE(M, UInt64, background_distributed_schedule_pool_size, 16) \
|
||||
MAKE_OBSOLETE(M, DefaultDatabaseEngine, default_database_engine, DefaultDatabaseEngine::Atomic) \
|
||||
MAKE_OBSOLETE(M, UInt64, max_pipeline_depth, 0) \
|
||||
|
||||
/** The section above is for obsolete settings. Do not add anything there. */
|
||||
|
||||
|
||||
|
@ -367,6 +367,27 @@ public:
|
||||
// Top-level OpenTelemetry trace context for the query. Makes sense only for a query context.
|
||||
OpenTelemetryTraceContext query_trace_context;
|
||||
|
||||
/// Some counters for current query execution.
|
||||
/// Most of them are workarounds and should be removed in the future.
|
||||
struct KitchenSink
|
||||
{
|
||||
std::atomic<size_t> analyze_counter = 0;
|
||||
|
||||
KitchenSink() = default;
|
||||
|
||||
KitchenSink(const KitchenSink & rhs)
|
||||
: analyze_counter(rhs.analyze_counter.load())
|
||||
{}
|
||||
|
||||
KitchenSink & operator=(const KitchenSink & rhs)
|
||||
{
|
||||
analyze_counter = rhs.analyze_counter.load();
|
||||
return *this;
|
||||
}
|
||||
};
|
||||
|
||||
KitchenSink kitchen_sink;
|
||||
|
||||
private:
|
||||
using SampleBlockCache = std::unordered_map<std::string, Block>;
|
||||
mutable SampleBlockCache sample_block_cache;
|
||||
|
@ -639,7 +639,18 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
analyze(shouldMoveToPrewhere());
|
||||
|
||||
bool need_analyze_again = false;
|
||||
if (analysis_result.prewhere_constant_filter_description.always_false || analysis_result.prewhere_constant_filter_description.always_true)
|
||||
bool can_analyze_again = false;
|
||||
if (context->hasQueryContext())
|
||||
{
|
||||
/// Check number of calls of 'analyze' function.
|
||||
/// If it is too big, we will not analyze the query again not to have exponential blowup.
|
||||
std::atomic<size_t> & current_query_analyze_count = context->getQueryContext()->kitchen_sink.analyze_counter;
|
||||
++current_query_analyze_count;
|
||||
can_analyze_again = settings.max_analyze_depth == 0 || current_query_analyze_count < settings.max_analyze_depth;
|
||||
}
|
||||
|
||||
if (can_analyze_again && (analysis_result.prewhere_constant_filter_description.always_false ||
|
||||
analysis_result.prewhere_constant_filter_description.always_true))
|
||||
{
|
||||
if (analysis_result.prewhere_constant_filter_description.always_true)
|
||||
query.setExpression(ASTSelectQuery::Expression::PREWHERE, {});
|
||||
@ -647,7 +658,9 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
query.setExpression(ASTSelectQuery::Expression::PREWHERE, std::make_shared<ASTLiteral>(0u));
|
||||
need_analyze_again = true;
|
||||
}
|
||||
if (analysis_result.where_constant_filter_description.always_false || analysis_result.where_constant_filter_description.always_true)
|
||||
|
||||
if (can_analyze_again && (analysis_result.where_constant_filter_description.always_false ||
|
||||
analysis_result.where_constant_filter_description.always_true))
|
||||
{
|
||||
if (analysis_result.where_constant_filter_description.always_true)
|
||||
query.setExpression(ASTSelectQuery::Expression::WHERE, {});
|
||||
@ -658,7 +671,8 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
|
||||
if (need_analyze_again)
|
||||
{
|
||||
LOG_TRACE(log, "Running 'analyze' second time");
|
||||
size_t current_query_analyze_count = context->getQueryContext()->kitchen_sink.analyze_counter.load();
|
||||
LOG_TRACE(log, "Running 'analyze' second time (current analyze depth: {})", current_query_analyze_count);
|
||||
|
||||
/// Reuse already built sets for multiple passes of analysis
|
||||
prepared_sets = query_analyzer->getPreparedSets();
|
||||
|
@ -144,6 +144,8 @@ ContextMutablePtr StorageNATS::addSettings(ContextPtr local_context) const
|
||||
modified_context->setSetting("input_format_skip_unknown_fields", true);
|
||||
modified_context->setSetting("input_format_allow_errors_ratio", 0.);
|
||||
modified_context->setSetting("input_format_allow_errors_num", nats_settings->nats_skip_broken_messages.value);
|
||||
/// Since we are reusing the same context for all queries executed simultaneously, we don't want to used shared `analyze_count`
|
||||
modified_context->setSetting("max_analyze_depth", Field{0});
|
||||
|
||||
if (!schema_name.empty())
|
||||
modified_context->setSetting("format_schema", schema_name);
|
||||
|
@ -242,6 +242,8 @@ ContextMutablePtr StorageRabbitMQ::addSettings(ContextPtr local_context) const
|
||||
modified_context->setSetting("input_format_skip_unknown_fields", true);
|
||||
modified_context->setSetting("input_format_allow_errors_ratio", 0.);
|
||||
modified_context->setSetting("input_format_allow_errors_num", rabbitmq_settings->rabbitmq_skip_broken_messages.value);
|
||||
/// Since we are reusing the same context for all queries executed simultaneously, we don't want to used shared `analyze_count`
|
||||
modified_context->setSetting("max_analyze_depth", Field{0});
|
||||
|
||||
if (!schema_name.empty())
|
||||
modified_context->setSetting("format_schema", schema_name);
|
||||
|
26
tests/queries/0_stateless/02337_join_analyze_stuck.reference
Normal file
26
tests/queries/0_stateless/02337_join_analyze_stuck.reference
Normal file
@ -0,0 +1,26 @@
|
||||
WITH
|
||||
x AS
|
||||
(
|
||||
SELECT number
|
||||
FROM numbers(10)
|
||||
),
|
||||
cross_sales AS
|
||||
(
|
||||
SELECT 1 AS xx
|
||||
FROM
|
||||
x,
|
||||
x AS d1,
|
||||
x AS d2,
|
||||
x AS d3,
|
||||
x AS d4,
|
||||
x AS d5,
|
||||
x AS d6,
|
||||
x AS d7,
|
||||
x AS d8,
|
||||
x AS d9
|
||||
WHERE x.number = d9.number
|
||||
)
|
||||
SELECT xx
|
||||
FROM
|
||||
cross_sales
|
||||
WHERE xx = 2000
|
13
tests/queries/0_stateless/02337_join_analyze_stuck.sql
Normal file
13
tests/queries/0_stateless/02337_join_analyze_stuck.sql
Normal file
@ -0,0 +1,13 @@
|
||||
-- Tags: long
|
||||
|
||||
-- https://github.com/ClickHouse/ClickHouse/issues/21557
|
||||
|
||||
EXPLAIN SYNTAX
|
||||
WITH
|
||||
x AS ( SELECT number FROM numbers(10) ),
|
||||
cross_sales AS (
|
||||
SELECT 1 AS xx
|
||||
FROM x, x AS d1, x AS d2, x AS d3, x AS d4, x AS d5, x AS d6, x AS d7, x AS d8, x AS d9
|
||||
WHERE x.number = d9.number
|
||||
)
|
||||
SELECT xx FROM cross_sales WHERE xx = 2000;
|
Loading…
Reference in New Issue
Block a user