mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Fix incorrect predicate push down with grouping sets (#46151)
This commit is contained in:
parent
2df52af445
commit
678e4250cd
@ -35,7 +35,8 @@ bool PredicateExpressionsOptimizer::optimize(ASTSelectQuery & select_query)
|
||||
if (!enable_optimize_predicate_expression)
|
||||
return false;
|
||||
|
||||
if (select_query.having() && (!select_query.group_by_with_cube && !select_query.group_by_with_rollup && !select_query.group_by_with_totals))
|
||||
const bool has_incompatible_constructs = select_query.group_by_with_cube || select_query.group_by_with_rollup || select_query.group_by_with_totals || select_query.group_by_with_grouping_sets;
|
||||
if (select_query.having() && !has_incompatible_constructs)
|
||||
tryMovePredicatesFromHavingToWhere(select_query);
|
||||
|
||||
if (!select_query.tables() || select_query.tables()->children.empty())
|
||||
|
@ -56,6 +56,8 @@ public:
|
||||
|
||||
const Aggregator::Params & getParams() const { return params; }
|
||||
|
||||
const auto & getGroupingSetsParamsList() const { return grouping_sets_params; }
|
||||
|
||||
bool inOrder() const { return !sort_description_for_merging.empty(); }
|
||||
bool explicitSortingRequired() const { return explicit_sorting_required_for_aggregation_in_order; }
|
||||
bool isGroupingSets() const { return !grouping_sets_params.empty(); }
|
||||
|
@ -53,6 +53,53 @@ static void checkChildrenSize(QueryPlan::Node * node, size_t child_num)
|
||||
child_num, child->getInputStreams().size(), node->children.size());
|
||||
}
|
||||
|
||||
static bool identifiersIsAmongAllGroupingSets(const GroupingSetsParamsList & grouping_sets_params, const NameSet & identifiers_in_predicate)
|
||||
{
|
||||
for (const auto & grouping_set : grouping_sets_params)
|
||||
{
|
||||
for (const auto & identifier : identifiers_in_predicate)
|
||||
{
|
||||
if (std::find(grouping_set.used_keys.begin(), grouping_set.used_keys.end(), identifier) == grouping_set.used_keys.end())
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
static NameSet findIdentifiersOfNode(const ActionsDAG::Node * node)
|
||||
{
|
||||
NameSet res;
|
||||
|
||||
/// We treat all INPUT as identifier
|
||||
if (node->type == ActionsDAG::ActionType::INPUT)
|
||||
{
|
||||
res.emplace(node->result_name);
|
||||
return res;
|
||||
}
|
||||
|
||||
std::queue<const ActionsDAG::Node *> queue;
|
||||
queue.push(node);
|
||||
|
||||
while (!queue.empty())
|
||||
{
|
||||
const auto * top = queue.front();
|
||||
for (const auto * child : top->children)
|
||||
{
|
||||
if (child->type == ActionsDAG::ActionType::INPUT)
|
||||
{
|
||||
res.emplace(child->result_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Only push non INPUT child into the queue
|
||||
queue.push(child);
|
||||
}
|
||||
}
|
||||
queue.pop();
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
static ActionsDAGPtr splitFilter(QueryPlan::Node * parent_node, const Names & allowed_inputs, size_t child_idx = 0)
|
||||
{
|
||||
QueryPlan::Node * child_node = parent_node->children.front();
|
||||
@ -176,6 +223,20 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
|
||||
|
||||
if (auto * aggregating = typeid_cast<AggregatingStep *>(child.get()))
|
||||
{
|
||||
/// If aggregating is GROUPING SETS, and not all the identifiers exist in all
|
||||
/// of the grouping sets, we could not push the filter down.
|
||||
if (aggregating->isGroupingSets())
|
||||
{
|
||||
|
||||
const auto & actions = filter->getExpression();
|
||||
const auto & filter_node = actions->findInOutputs(filter->getFilterColumnName());
|
||||
|
||||
auto identifiers_in_predicate = findIdentifiersOfNode(&filter_node);
|
||||
|
||||
if (!identifiersIsAmongAllGroupingSets(aggregating->getGroupingSetsParamsList(), identifiers_in_predicate))
|
||||
return 0;
|
||||
}
|
||||
|
||||
const auto & params = aggregating->getParams();
|
||||
const auto & keys = params.keys;
|
||||
|
||||
|
@ -0,0 +1,62 @@
|
||||
---Explain Syntax---
|
||||
SELECT
|
||||
day_,
|
||||
type_1
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
day_,
|
||||
if(type_1 = \'\', \'all\', type_1) AS type_1
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
day_,
|
||||
type_1
|
||||
FROM test_grouping_sets_predicate
|
||||
PREWHERE day_ = \'2023-01-05\'
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(day_, type_1),
|
||||
(day_))
|
||||
HAVING if(type_1 = \'\', \'all\', type_1) = \'all\'
|
||||
) AS t
|
||||
WHERE type_1 = \'all\'
|
||||
)
|
||||
WHERE type_1 = \'all\'
|
||||
|
||||
---Explain Pipeline---
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
(Filter)
|
||||
FilterTransform × 2
|
||||
(Filter)
|
||||
FilterTransform × 2
|
||||
(Filter)
|
||||
FilterTransform × 2
|
||||
(Aggregating)
|
||||
ExpressionTransform × 2
|
||||
AggregatingTransform × 2
|
||||
Copy 1 → 2
|
||||
(Expression)
|
||||
ExpressionTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder 0 → 1
|
||||
|
||||
---Result---
|
||||
2023-01-05 all
|
||||
|
||||
---Explain Pipeline---
|
||||
(Expression)
|
||||
ExpressionTransform × 2
|
||||
(Aggregating)
|
||||
ExpressionTransform × 2
|
||||
AggregatingTransform × 2
|
||||
Copy 1 → 2
|
||||
(Filter)
|
||||
FilterTransform
|
||||
(Filter)
|
||||
FilterTransform
|
||||
(Filter)
|
||||
FilterTransform
|
||||
(ReadFromMergeTree)
|
||||
MergeTreeInOrder 0 → 1
|
@ -0,0 +1,109 @@
|
||||
DROP TABLE IF EXISTS test_grouping_sets_predicate;
|
||||
|
||||
CREATE TABLE test_grouping_sets_predicate
|
||||
(
|
||||
day_ Date,
|
||||
type_1 String
|
||||
)
|
||||
ENGINE=MergeTree
|
||||
ORDER BY day_;
|
||||
|
||||
INSERT INTO test_grouping_sets_predicate SELECT
|
||||
toDate('2023-01-05') AS day_,
|
||||
'hello, world'
|
||||
FROM numbers (10);
|
||||
|
||||
SELECT '---Explain Syntax---';
|
||||
EXPLAIN SYNTAX
|
||||
SELECT *
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
day_,
|
||||
if(type_1 = '', 'all', type_1) AS type_1
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
day_,
|
||||
type_1
|
||||
FROM test_grouping_sets_predicate
|
||||
WHERE day_ = '2023-01-05'
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(day_, type_1),
|
||||
(day_))
|
||||
) AS t
|
||||
)
|
||||
WHERE type_1 = 'all';
|
||||
|
||||
SELECT '';
|
||||
SELECT '---Explain Pipeline---';
|
||||
EXPLAIN PIPELINE
|
||||
SELECT *
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
day_,
|
||||
if(type_1 = '', 'all', type_1) AS type_1
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
day_,
|
||||
type_1
|
||||
FROM test_grouping_sets_predicate
|
||||
WHERE day_ = '2023-01-05'
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(day_, type_1),
|
||||
(day_))
|
||||
) AS t
|
||||
)
|
||||
WHERE type_1 = 'all';
|
||||
|
||||
SELECT '';
|
||||
SELECT '---Result---';
|
||||
SELECT *
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
day_,
|
||||
if(type_1 = '', 'all', type_1) AS type_1
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
day_,
|
||||
type_1
|
||||
FROM test_grouping_sets_predicate
|
||||
WHERE day_ = '2023-01-05'
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(day_, type_1),
|
||||
(day_))
|
||||
) AS t
|
||||
)
|
||||
WHERE type_1 = 'all';
|
||||
|
||||
SELECT '';
|
||||
SELECT '---Explain Pipeline---';
|
||||
EXPLAIN PIPELINE
|
||||
SELECT *
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
day_,
|
||||
if(type_1 = '', 'all', type_1) AS type_1
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
day_,
|
||||
type_1
|
||||
FROM test_grouping_sets_predicate
|
||||
GROUP BY
|
||||
GROUPING SETS (
|
||||
(day_, type_1),
|
||||
(day_))
|
||||
) AS t
|
||||
)
|
||||
WHERE day_ = '2023-01-05';
|
||||
|
||||
DROP TABLE test_grouping_sets_predicate;
|
Loading…
Reference in New Issue
Block a user