Skip to content

Commit

Permalink
Merge pull request ClickHouse#63405 from ClickHouse/vdimir/group_by_u…
Browse files Browse the repository at this point in the history
…se_nulls_fixes

Small fixes for group_by_use_nulls
  • Loading branch information
vdimir authored May 14, 2024
2 parents f5decc7 + f070462 commit 6f74beb
Show file tree
Hide file tree
Showing 8 changed files with 89 additions and 2 deletions.
16 changes: 16 additions & 0 deletions src/Analyzer/Passes/QueryAnalysisPass.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -4835,6 +4835,19 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I
}
}

if (!scope.expressions_in_resolve_process_stack.hasAggregateFunction())
{
for (auto & [node, _] : matched_expression_nodes_with_names)
{
auto it = scope.nullable_group_by_keys.find(node);
if (it != scope.nullable_group_by_keys.end())
{
node = it->node->clone();
node->convertToNullable();
}
}
}

std::unordered_map<const IColumnTransformerNode *, std::unordered_set<std::string>> strict_transformer_to_used_column_names;
for (const auto & transformer : matcher_node_typed.getColumnTransformers().getNodes())
{
Expand Down Expand Up @@ -5028,7 +5041,10 @@ ProjectionNames QueryAnalyzer::resolveMatcher(QueryTreeNodePtr & matcher_node, I
scope.scope_node->formatASTForErrorMessage());
}

auto original_ast = matcher_node->getOriginalAST();
matcher_node = std::move(list);
if (original_ast)
matcher_node->setOriginalAST(original_ast);

return result_projection_names;
}
Expand Down
4 changes: 4 additions & 0 deletions src/Analyzer/ValidationUtils.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,10 @@ namespace

void validateFilter(const QueryTreeNodePtr & filter_node, std::string_view exception_place_message, const QueryTreeNodePtr & query_node)
{
if (filter_node->getNodeType() == QueryTreeNodeType::LIST)
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Unsupported expression '{}' in filter", filter_node->formatASTForErrorMessage());

auto filter_node_result_type = filter_node->getResultType();
if (!filter_node_result_type->canBeUsedInBooleanContext())
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER,
Expand Down
2 changes: 1 addition & 1 deletion src/Functions/FunctionHelpers.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ ColumnWithTypeAndName columnGetNested(const ColumnWithTypeAndName & col)
return ColumnWithTypeAndName{ nullable_res, nested_type, col.name };
}
else
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column for DataTypeNullable");
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} for DataTypeNullable", col.dumpStructure());
}
return col;
}
Expand Down
1 change: 1 addition & 0 deletions src/Processors/QueryPlan/AggregatingStep.h
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@ class AggregatingStep : public ITransformingStep
const Aggregator::Params & getParams() const { return params; }

const auto & getGroupingSetsParamsList() const { return grouping_sets_params; }
bool isGroupByUseNulls() const { return group_by_use_nulls; }

bool inOrder() const { return !sort_description_for_merging.empty(); }
bool explicitSortingRequired() const { return explicit_sorting_required_for_aggregation_in_order; }
Expand Down
3 changes: 3 additions & 0 deletions src/Processors/QueryPlan/Optimizations/filterPushDown.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -428,6 +428,9 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
/// of the grouping sets, we could not push the filter down.
if (aggregating->isGroupingSets())
{
/// Cannot push down filter if type has been changed.
if (aggregating->isGroupByUseNulls())
return 0;

const auto & actions = filter->getExpression();
const auto & filter_node = actions->findInOutputs(filter->getFilterColumnName());
Expand Down
2 changes: 1 addition & 1 deletion tests/queries/0_stateless/03101_analyzer_identifiers_3.sql
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ SELECT * GROUP BY *;
-- not ok as every component of ORDER BY may contain ASC/DESC and COLLATE; though can be supported in some sense
-- but it works
SELECT * ORDER BY *;
SELECT * WHERE *; -- { serverError UNSUPPORTED_METHOD }
SELECT * WHERE *; -- { serverError BAD_ARGUMENTS }

SELECT '---';

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
2023-01-05 hello, world
2023-01-05 \N
2023-01-05 hello, world
2023-01-05 \N
2023-01-05
2023-01-05
2023-01-05 hello, world
2023-01-05 \N
2023-01-05 hello, world
2023-01-05 \N
2023-01-05 10
2023-01-05 10
2023-01-05 hello, world
2023-01-05 \N
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
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);

SET group_by_use_nulls = true;

SELECT *
FROM ( SELECT day_, type_1 FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) )
WHERE day_ = '2023-01-05'
ORDER BY ALL;


SELECT *
FROM ( SELECT * FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) )
WHERE day_ = '2023-01-05'
ORDER BY ALL;

SELECT *
FROM ( SELECT day_ FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) )
WHERE day_ = '2023-01-05'
ORDER BY *
SETTINGS allow_experimental_analyzer=1;

SELECT *
FROM ( SELECT * FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) )
WHERE day_ = '2023-01-05'
GROUP BY *
ORDER BY ALL
SETTINGS allow_experimental_analyzer=1;

SELECT *
FROM ( SELECT * FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (*), (day_) ) )
WHERE day_ = '2023-01-05'
GROUP BY GROUPING SETS (*)
ORDER BY type_1
SETTINGS allow_experimental_analyzer=1;

SELECT *
FROM ( SELECT day_, COUNT(*) FROM test_grouping_sets_predicate GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) )
WHERE day_ = '2023-01-05'
ORDER BY ALL;


SELECT t2.*
FROM ( SELECT t1.* FROM test_grouping_sets_predicate t1 GROUP BY GROUPING SETS ( (day_, type_1), (day_) ) ) t2
WHERE day_ = '2023-01-05'
ORDER BY ALL;

0 comments on commit 6f74beb

Please sign in to comment.