Merge pull request #40334 from ClickHouse/vdimir/analyze-stuck-limit

Limit number of analyze for one query, att. 2
This commit is contained in:
Vladimir C 2022-09-01 14:40:20 +02:00 committed by GitHub
commit f2cf7d7762
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 84 additions and 4 deletions

View File

@ -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. */

View File

@ -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;

View File

@ -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();

View File

@ -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);

View File

@ -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);

View 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

View 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;