mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 13:02:00 +00:00
Review fixes
This commit is contained in:
parent
5a51d1c29b
commit
a1a2e276ae
@ -100,7 +100,7 @@ bool checkPositionalArguments(ASTPtr & argument, const ASTSelectQuery * select_q
|
||||
{
|
||||
auto columns = select_query->select()->children;
|
||||
|
||||
/// In case of expression/function (order by 1+2 and 2*x1, max(1, 2)) replace
|
||||
/// In case of expression/function (order by 1+2 and 2*x1, greatest(1, 2)) replace
|
||||
/// positions only if all literals are numbers, otherwise it is not positional.
|
||||
bool positional = true;
|
||||
|
||||
@ -120,22 +120,47 @@ bool checkPositionalArguments(ASTPtr & argument, const ASTSelectQuery * select_q
|
||||
}
|
||||
else if (const auto * function_ast = typeid_cast<const ASTFunction *>(column.get()))
|
||||
{
|
||||
auto is_aggregate_function = AggregateFunctionFactory::instance().isAggregateFunctionName(function_ast->name);
|
||||
if (is_aggregate_function && expression != ASTSelectQuery::Expression::ORDER_BY)
|
||||
std::function<void(ASTPtr)> throw_if_aggregate_function = [&](ASTPtr node)
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal value (aggregate function) for positional argument in {}",
|
||||
ASTSelectQuery::expressionToString(expression));
|
||||
}
|
||||
if (const auto * function = typeid_cast<const ASTFunction *>(node.get()))
|
||||
{
|
||||
auto is_aggregate_function = AggregateFunctionFactory::instance().isAggregateFunctionName(function->name);
|
||||
if (is_aggregate_function)
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal value (aggregate function) for positional argument in {}",
|
||||
ASTSelectQuery::expressionToString(expression));
|
||||
}
|
||||
else
|
||||
{
|
||||
if (function->arguments)
|
||||
{
|
||||
for (const auto & arg : function->arguments->children)
|
||||
throw_if_aggregate_function(arg);
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
if (expression == ASTSelectQuery::Expression::GROUP_BY)
|
||||
throw_if_aggregate_function(column);
|
||||
|
||||
argument = column->clone();
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal value for positional argument in {}",
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal value for positional argument in {}",
|
||||
ASTSelectQuery::expressionToString(expression));
|
||||
}
|
||||
}
|
||||
/// Do not throw if out of bounds, see appendUnusedGroupByColumn.
|
||||
else if (pos > columns.size() || !pos)
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Positional argument out of bounds: {} (exprected in range [1, {}]",
|
||||
pos, columns.size());
|
||||
}
|
||||
/// Do not throw if pos < 0, becuase of TreeOptimizer::appendUnusedColumn()
|
||||
}
|
||||
else
|
||||
positional = false;
|
||||
|
@ -67,26 +67,17 @@ const std::unordered_set<String> possibly_injective_function_names
|
||||
* Instead, leave `GROUP BY const`.
|
||||
* Next, see deleting the constants in the analyzeAggregation method.
|
||||
*/
|
||||
void appendUnusedGroupByColumn(ASTSelectQuery * select_query, const NameSet & source_columns)
|
||||
void appendUnusedGroupByColumn(ASTSelectQuery * select_query)
|
||||
{
|
||||
/// You must insert a constant that is not the name of the column in the table. Such a case is rare, but it happens.
|
||||
/// Also start unused_column integer from source_columns.size() + 1, because lower numbers ([1, source_columns.size()])
|
||||
/// Also start unused_column integer must not intersect with ([1, source_columns.size()])
|
||||
/// might be in positional GROUP BY.
|
||||
UInt64 unused_column = source_columns.size() + 1;
|
||||
String unused_column_name = toString(unused_column);
|
||||
|
||||
while (source_columns.count(unused_column_name))
|
||||
{
|
||||
++unused_column;
|
||||
unused_column_name = toString(unused_column);
|
||||
}
|
||||
|
||||
select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, std::make_shared<ASTExpressionList>());
|
||||
select_query->groupBy()->children.emplace_back(std::make_shared<ASTLiteral>(UInt64(unused_column)));
|
||||
select_query->groupBy()->children.emplace_back(std::make_shared<ASTLiteral>(Int64(-1)));
|
||||
}
|
||||
|
||||
/// Eliminates injective function calls and constant expressions from group by statement.
|
||||
void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_columns, ContextPtr context)
|
||||
void optimizeGroupBy(ASTSelectQuery * select_query, ContextPtr context)
|
||||
{
|
||||
const FunctionFactory & function_factory = FunctionFactory::instance();
|
||||
|
||||
@ -191,7 +182,7 @@ void optimizeGroupBy(ASTSelectQuery * select_query, const NameSet & source_colum
|
||||
}
|
||||
|
||||
if (group_exprs.empty())
|
||||
appendUnusedGroupByColumn(select_query, source_columns);
|
||||
appendUnusedGroupByColumn(select_query);
|
||||
}
|
||||
|
||||
struct GroupByKeysInfo
|
||||
@ -710,7 +701,7 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result,
|
||||
optimizeAggregationFunctions(query);
|
||||
|
||||
/// GROUP BY injective function elimination.
|
||||
optimizeGroupBy(select_query, result.source_columns_set, context);
|
||||
optimizeGroupBy(select_query, context);
|
||||
|
||||
/// GROUP BY functions of other keys elimination.
|
||||
if (settings.optimize_group_by_function_keys)
|
||||
|
@ -100,7 +100,16 @@ GROUP BY x2
|
||||
ORDER BY
|
||||
max(x1) ASC,
|
||||
x2 ASC
|
||||
explain syntax select 1 + greatest(x1, 1), x2 from test group by 1, 2;
|
||||
SELECT
|
||||
1 + greatest(x1, 1),
|
||||
x2
|
||||
FROM test
|
||||
GROUP BY
|
||||
1 + greatest(x1, 1),
|
||||
x2
|
||||
select max(x1), x2 from test group by 1, 2; -- { serverError 43 }
|
||||
select 1 + max(x1), x2 from test group by 1, 2; -- { serverError 43 }
|
||||
select x1 + x2, x3 from test group by x1 + x2, x3;
|
||||
11 100
|
||||
200 1
|
||||
|
@ -33,8 +33,10 @@ explain syntax select x3 + 1, x2, x1 from test order by 1;
|
||||
explain syntax select x3, x3 - x2, x2, x1 from test order by 2;
|
||||
explain syntax select x3, if(x3 > 10, x3, plus(x1, x2)), x1 + x2 from test order by 2;
|
||||
explain syntax select max(x1), x2 from test group by 2 order by 1, 2;
|
||||
explain syntax select 1 + greatest(x1, 1), x2 from test group by 1, 2;
|
||||
|
||||
select max(x1), x2 from test group by 1, 2; -- { serverError 43 }
|
||||
select 1 + max(x1), x2 from test group by 1, 2; -- { serverError 43 }
|
||||
select x1 + x2, x3 from test group by x1 + x2, x3;
|
||||
|
||||
select x3, x2, x1 from test order by x3 * 2, x2, x1; -- check x3 * 2 does not become x3 * x2
|
||||
|
Loading…
Reference in New Issue
Block a user