mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-18 12:22:12 +00:00
Backport #66126 to 24.6: Disable merge filters optimization by default.
This commit is contained in:
parent
e68d91cb01
commit
8a4c207721
@ -756,6 +756,7 @@ class IColumn;
|
||||
M(Bool, query_plan_push_down_limit, true, "Allow to move LIMITs down in the query plan", 0) \
|
||||
M(Bool, query_plan_split_filter, true, "Allow to split filters in the query plan", 0) \
|
||||
M(Bool, query_plan_merge_expressions, true, "Allow to merge expressions in the query plan", 0) \
|
||||
M(Bool, query_plan_merge_filters, false, "Allow to merge filters in the query plan", 0) \
|
||||
M(Bool, query_plan_filter_push_down, true, "Allow to push down filter by predicate query plan step", 0) \
|
||||
M(Bool, query_plan_convert_outer_join_to_inner_join, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values", 0) \
|
||||
M(Bool, query_plan_optimize_prewhere, true, "Allow to push down filter to PREWHERE expression for supported storages", 0) \
|
||||
|
@ -46,6 +46,10 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes);
|
||||
/// Replace chain `FilterStep -> ExpressionStep` to single FilterStep
|
||||
size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &);
|
||||
|
||||
/// Replace chain `FilterStep -> FilterStep` to single FilterStep
|
||||
/// Note: this breaks short-circuit logic, so it is disabled for now.
|
||||
size_t tryMergeFilters(QueryPlan::Node * parent_node, QueryPlan::Nodes &);
|
||||
|
||||
/// Move FilterStep down if possible.
|
||||
/// May split FilterStep and push down only part of it.
|
||||
size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes);
|
||||
@ -81,11 +85,12 @@ size_t tryAggregatePartitionsIndependently(QueryPlan::Node * node, QueryPlan::No
|
||||
|
||||
inline const auto & getOptimizations()
|
||||
{
|
||||
static const std::array<Optimization, 11> optimizations = {{
|
||||
static const std::array<Optimization, 12> optimizations = {{
|
||||
{tryLiftUpArrayJoin, "liftUpArrayJoin", &QueryPlanOptimizationSettings::lift_up_array_join},
|
||||
{tryPushDownLimit, "pushDownLimit", &QueryPlanOptimizationSettings::push_down_limit},
|
||||
{trySplitFilter, "splitFilter", &QueryPlanOptimizationSettings::split_filter},
|
||||
{tryMergeExpressions, "mergeExpressions", &QueryPlanOptimizationSettings::merge_expressions},
|
||||
{tryMergeFilters, "mergeFilters", &QueryPlanOptimizationSettings::merge_filters},
|
||||
{tryPushDownFilter, "pushDownFilter", &QueryPlanOptimizationSettings::filter_push_down},
|
||||
{tryConvertOuterJoinToInnerJoin, "convertOuterJoinToInnerJoin", &QueryPlanOptimizationSettings::convert_outer_join_to_inner_join},
|
||||
{tryExecuteFunctionsAfterSorting, "liftUpFunctions", &QueryPlanOptimizationSettings::execute_functions_after_sorting},
|
||||
|
@ -20,6 +20,8 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const
|
||||
|
||||
settings.merge_expressions = from.query_plan_enable_optimizations && from.query_plan_merge_expressions;
|
||||
|
||||
settings.merge_filters = from.query_plan_enable_optimizations && from.query_plan_merge_filters;
|
||||
|
||||
settings.filter_push_down = from.query_plan_enable_optimizations && from.query_plan_filter_push_down;
|
||||
|
||||
settings.convert_outer_join_to_inner_join = from.query_plan_enable_optimizations && from.query_plan_convert_outer_join_to_inner_join;
|
||||
|
@ -31,6 +31,9 @@ struct QueryPlanOptimizationSettings
|
||||
/// If merge-expressions optimization is enabled.
|
||||
bool merge_expressions = true;
|
||||
|
||||
/// If merge-filters optimization is enabled.
|
||||
bool merge_filters = false;
|
||||
|
||||
/// If filter push down optimization is enabled.
|
||||
bool filter_push_down = true;
|
||||
|
||||
|
@ -34,7 +34,6 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
|
||||
auto * parent_expr = typeid_cast<ExpressionStep *>(parent.get());
|
||||
auto * parent_filter = typeid_cast<FilterStep *>(parent.get());
|
||||
auto * child_expr = typeid_cast<ExpressionStep *>(child.get());
|
||||
auto * child_filter = typeid_cast<FilterStep *>(child.get());
|
||||
|
||||
if (parent_expr && child_expr)
|
||||
{
|
||||
@ -76,7 +75,23 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
|
||||
parent_node->children.swap(child_node->children);
|
||||
return 1;
|
||||
}
|
||||
else if (parent_filter && child_filter)
|
||||
|
||||
return 0;
|
||||
}
|
||||
size_t tryMergeFilters(QueryPlan::Node * parent_node, QueryPlan::Nodes &)
|
||||
{
|
||||
if (parent_node->children.size() != 1)
|
||||
return false;
|
||||
|
||||
QueryPlan::Node * child_node = parent_node->children.front();
|
||||
|
||||
auto & parent = parent_node->step;
|
||||
auto & child = child_node->step;
|
||||
|
||||
auto * parent_filter = typeid_cast<FilterStep *>(parent.get());
|
||||
auto * child_filter = typeid_cast<FilterStep *>(child.get());
|
||||
|
||||
if (parent_filter && child_filter)
|
||||
{
|
||||
const auto & child_actions = child_filter->getExpression();
|
||||
const auto & parent_actions = parent_filter->getExpression();
|
||||
|
@ -163,6 +163,7 @@ Filter column: notEquals(__table1.y, 2_UInt8)
|
||||
> filter is pushed down before CreatingSets
|
||||
CreatingSets
|
||||
Filter
|
||||
Filter
|
||||
1
|
||||
3
|
||||
> one condition of filter is pushed down before LEFT JOIN
|
||||
|
@ -1,3 +1,5 @@
|
||||
set query_plan_merge_filters=1;
|
||||
|
||||
set allow_experimental_analyzer=1;
|
||||
select explain from (explain actions = 1 select * from (select sum(number) as v, bitAnd(number, 15) as key from numbers(1e8) group by key having v != 0) where key = 7) where explain like '%Filter%' or explain like '%Aggregating%';
|
||||
|
||||
|
@ -332,12 +332,13 @@ SETTINGS optimize_aggregators_of_group_by_keys=0 -- avoid removing any() as it d
|
||||
Expression (Projection)
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression (Before ORDER BY)
|
||||
Filter (((WHERE + (Projection + Before ORDER BY)) + HAVING))
|
||||
Aggregating
|
||||
Expression ((Before GROUP BY + Projection))
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression ((Before ORDER BY + (Projection + Before ORDER BY)))
|
||||
ReadFromSystemNumbers
|
||||
Filter ((WHERE + (Projection + Before ORDER BY)))
|
||||
Filter (HAVING)
|
||||
Aggregating
|
||||
Expression ((Before GROUP BY + Projection))
|
||||
Sorting (Sorting for ORDER BY)
|
||||
Expression ((Before ORDER BY + (Projection + Before ORDER BY)))
|
||||
ReadFromSystemNumbers
|
||||
-- execute
|
||||
1
|
||||
2
|
||||
|
@ -29,16 +29,20 @@ WHERE type_1 = \'all\'
|
||||
ExpressionTransform × 2
|
||||
(Filter)
|
||||
FilterTransform × 2
|
||||
(Aggregating)
|
||||
ExpressionTransform × 2
|
||||
AggregatingTransform × 2
|
||||
Copy 1 → 2
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
(Filter)
|
||||
FilterTransform × 2
|
||||
(Filter)
|
||||
FilterTransform × 2
|
||||
(Aggregating)
|
||||
ExpressionTransform × 2
|
||||
AggregatingTransform × 2
|
||||
Copy 1 → 2
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
(Filter)
|
||||
@ -64,10 +68,14 @@ ExpressionTransform × 2
|
||||
ExpressionTransform × 2
|
||||
AggregatingTransform × 2
|
||||
Copy 1 → 2
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
(Filter)
|
||||
FilterTransform
|
||||
(Filter)
|
||||
FilterTransform
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeSelect(pool: ReadPoolInOrder, algorithm: InOrder) 0 → 1
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
(Aggregating)
|
||||
|
70
tests/queries/0_stateless/03199_merge_filters_bug.sql
Normal file
70
tests/queries/0_stateless/03199_merge_filters_bug.sql
Normal file
@ -0,0 +1,70 @@
|
||||
drop table if exists t1;
|
||||
drop table if exists t2;
|
||||
|
||||
CREATE TABLE t1
|
||||
(
|
||||
`s1` String,
|
||||
`s2` String,
|
||||
`s3` String
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY tuple();
|
||||
|
||||
|
||||
CREATE TABLE t2
|
||||
(
|
||||
`fs1` FixedString(10),
|
||||
`fs2` FixedString(10)
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY tuple();
|
||||
|
||||
INSERT INTO t1 SELECT
|
||||
repeat('t', 15) s1,
|
||||
'test' s2,
|
||||
'test' s3;
|
||||
|
||||
INSERT INTO t1 SELECT
|
||||
substring(s1, 1, 10),
|
||||
s2,
|
||||
s3
|
||||
FROM generateRandom('s1 String, s2 String, s3 String')
|
||||
LIMIT 10000;
|
||||
|
||||
INSERT INTO t2 SELECT *
|
||||
FROM generateRandom()
|
||||
LIMIT 10000;
|
||||
|
||||
WITH
|
||||
tmp1 AS
|
||||
(
|
||||
SELECT
|
||||
CAST(s1, 'FixedString(10)') AS fs1,
|
||||
s2 AS sector,
|
||||
s3
|
||||
FROM t1
|
||||
WHERE (s3 != 'test')
|
||||
)
|
||||
SELECT
|
||||
fs1
|
||||
FROM t2
|
||||
LEFT JOIN tmp1 USING (fs1)
|
||||
WHERE (fs1 IN ('test')) SETTINGS enable_multiple_prewhere_read_steps = 0;
|
||||
|
||||
optimize table t1 final;
|
||||
|
||||
WITH
|
||||
tmp1 AS
|
||||
(
|
||||
SELECT
|
||||
CAST(s1, 'FixedString(10)') AS fs1,
|
||||
s2 AS sector,
|
||||
s3
|
||||
FROM t1
|
||||
WHERE (s3 != 'test')
|
||||
)
|
||||
SELECT
|
||||
fs1
|
||||
FROM t2
|
||||
LEFT JOIN tmp1 USING (fs1)
|
||||
WHERE (fs1 IN ('test'));
|
Loading…
Reference in New Issue
Block a user