Don't allow Variant/Dynamic types in ORDER BY/GROUP BY/PARTITION BY/PRIMARY KEY by default

This commit is contained in:
avogar 2024-09-18 14:10:03 +00:00
parent a7d0a5991e
commit dd6503bb2b
11 changed files with 472 additions and 5 deletions

View File

@ -5682,3 +5682,25 @@ Default value: `0`.
Enable `IF NOT EXISTS` for `CREATE` statement by default. If either this setting or `IF NOT EXISTS` is specified and a table with the provided name already exists, no exception will be thrown.
Default value: `false`.
## allow_suspicious_types_in_group_by {#allow_suspicious_types_in_group_by}
Allows or restricts using [Variant](../../sql-reference/data-types/variant.md) and [Dynamic](../../sql-reference/data-types/dynamic.md) types in GROUP BY keys.
Possible values:
- 1 — Usage of `Variant` and `Dynamic` types is not restricted.
- 0 — Usage of `Variant` and `Dynamic` types is restricted.
Default value: 0.
## allow_suspicious_types_in_group_by {#allow_suspicious_types_in_group_by}
Allows or restricts using [Variant](../../sql-reference/data-types/variant.md) and [Dynamic](../../sql-reference/data-types/dynamic.md) types in GROUP BY keys.
Possible values:
- 1 — Usage of `Variant` and `Dynamic` types is not restricted.
- 0 — Usage of `Variant` and `Dynamic` types is restricted.
Default value: 0.

View File

@ -411,6 +411,9 @@ SELECT d, dynamicType(d) FROM test ORDER by d;
└─────┴────────────────┘
```
**Note** by default `Dynamic` type is not allowed in `GROUP BY`/`ORDER BY` keys, if you want to use it consider its special comparison rule and enable `allow_suspicious_types_in_group_by`/`allow_suspicious_types_in_order_by` settings.
## Reaching the limit in number of different data types stored inside Dynamic
`Dynamic` data type can store only limited number of different data types as separate subcolumns. By default, this limit is 32, but you can change it in type declaration using syntax `Dynamic(max_types=N)` where N is between 0 and 254 (due to implementation details, it's impossible to have more than 254 different data types that can be stored as separate subcolumns inside Dynamic).

View File

@ -441,6 +441,8 @@ SELECT v, variantType(v) FROM test ORDER by v;
└─────┴────────────────┘
```
**Note** by default `Variant` type is not allowed in `GROUP BY`/`ORDER BY` keys, if you want to use it consider its special comparison rule and enable `allow_suspicious_types_in_group_by`/`allow_suspicious_types_in_order_by` settings.
## JSONExtract functions with Variant
All `JSONExtract*` functions support `Variant` type:

View File

@ -3962,6 +3962,8 @@ ProjectionNames QueryAnalyzer::resolveSortNodeList(QueryTreeNodePtr & sort_node_
sort_node.getExpression() = sort_column_list_node->getNodes().front();
}
validateSortingKeyType(sort_node.getExpression()->getResultType(), scope);
size_t sort_expression_projection_names_size = sort_expression_projection_names.size();
if (sort_expression_projection_names_size != 1)
throw Exception(ErrorCodes::LOGICAL_ERROR,
@ -4047,6 +4049,24 @@ ProjectionNames QueryAnalyzer::resolveSortNodeList(QueryTreeNodePtr & sort_node_
return result_projection_names;
}
void QueryAnalyzer::validateSortingKeyType(const DataTypePtr & sorting_key_type, const IdentifierResolveScope & scope) const
{
if (scope.context->getSettingsRef().allow_suspicious_types_in_order_by)
return;
auto check = [](const IDataType & type)
{
if (isDynamic(type) || isVariant(type))
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Data types Variant/Dynamic are not allowed in ORDER BY keys, because it can lead to unexpected results. "
"Set setting allow_suspicious_types_in_order_by = 1 in order to allow it");
};
check(*sorting_key_type);
sorting_key_type->forEachChild(check);
}
namespace
{
@ -4086,11 +4106,12 @@ void QueryAnalyzer::resolveGroupByNode(QueryNode & query_node_typed, IdentifierR
expandTuplesInList(group_by_list);
}
if (scope.group_by_use_nulls)
for (const auto & grouping_set : query_node_typed.getGroupBy().getNodes())
{
for (const auto & grouping_set : query_node_typed.getGroupBy().getNodes())
for (const auto & group_by_elem : grouping_set->as<ListNode>()->getNodes())
{
for (const auto & group_by_elem : grouping_set->as<ListNode>()->getNodes())
validateGroupByKeyType(group_by_elem->getResultType(), scope);
if (scope.group_by_use_nulls)
scope.nullable_group_by_keys.insert(group_by_elem);
}
}
@ -4106,14 +4127,35 @@ void QueryAnalyzer::resolveGroupByNode(QueryNode & query_node_typed, IdentifierR
auto & group_by_list = query_node_typed.getGroupBy().getNodes();
expandTuplesInList(group_by_list);
if (scope.group_by_use_nulls)
for (const auto & group_by_elem : query_node_typed.getGroupBy().getNodes())
{
for (const auto & group_by_elem : query_node_typed.getGroupBy().getNodes())
validateGroupByKeyType(group_by_elem->getResultType(), scope);
if (scope.group_by_use_nulls)
scope.nullable_group_by_keys.insert(group_by_elem);
}
}
}
/** Validate data types of GROUP BY key.
*/
void QueryAnalyzer::validateGroupByKeyType(const DataTypePtr & group_by_key_type, const IdentifierResolveScope & scope) const
{
if (scope.context->getSettingsRef().allow_suspicious_types_in_group_by)
return;
auto check = [](const IDataType & type)
{
if (isDynamic(type) || isVariant(type))
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Data types Variant/Dynamic are not allowed in GROUP BY keys, because it can lead to unexpected results. "
"Set setting allow_suspicious_types_in_group_by = 1 in order to allow it");
};
check(*group_by_key_type);
group_by_key_type->forEachChild(check);
}
/** Resolve interpolate columns nodes list.
*/
void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpolate_node_list, IdentifierResolveScope & scope)

View File

@ -217,8 +217,12 @@ private:
ProjectionNames resolveSortNodeList(QueryTreeNodePtr & sort_node_list, IdentifierResolveScope & scope);
void validateSortingKeyType(const DataTypePtr & sorting_key_type, const IdentifierResolveScope & scope) const;
void resolveGroupByNode(QueryNode & query_node_typed, IdentifierResolveScope & scope);
void validateGroupByKeyType(const DataTypePtr & group_by_key_type, const IdentifierResolveScope & scope) const;
void resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpolate_node_list, IdentifierResolveScope & scope);
void resolveWindowNodeList(QueryTreeNodePtr & window_node_list, IdentifierResolveScope & scope);

View File

@ -389,6 +389,9 @@ class IColumn;
M(Bool, prefer_global_in_and_join, false, "If enabled, all IN/JOIN operators will be rewritten as GLOBAL IN/JOIN. It's useful when the to-be-joined tables are only available on the initiator and we need to always scatter their data on-the-fly during distributed processing with the GLOBAL keyword. It's also useful to reduce the need to access the external sources joining external tables.", 0) \
M(Bool, enable_vertical_final, true, "If enable, remove duplicated rows during FINAL by marking rows as deleted and filtering them later instead of merging rows", 0) \
\
M(Bool, allow_suspicious_types_in_group_by, false, "Allow suspicious types like Variant/Dynamic in GROUP BY clause", 0) \
M(Bool, allow_suspicious_types_in_order_by, false, "Allow suspicious types like Variant/Dynamic in ORDER BY clause", 0) \
\
\
/** Limits during query execution are part of the settings. \
* Used to provide a more safe execution of queries from the user interface. \

View File

@ -1367,6 +1367,9 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain
}
}
for (const auto & result_column : step.getResultColumns())
validateGroupByKeyType(result_column.type);
if (optimize_aggregation_in_order)
{
for (auto & child : asts)
@ -1381,6 +1384,24 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain
return true;
}
void SelectQueryExpressionAnalyzer::validateGroupByKeyType(const DB::DataTypePtr & key_type) const
{
if (getContext()->getSettingsRef().allow_suspicious_types_in_group_by)
return;
auto check = [](const IDataType & type)
{
if (isDynamic(type) || isVariant(type))
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Data types Variant/Dynamic are not allowed in GROUP BY keys, because it can lead to unexpected results. "
"Set setting allow_suspicious_types_in_group_by = 1 in order to allow it");
};
check(*key_type);
key_type->forEachChild(check);
}
void SelectQueryExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types)
{
const auto * select_query = getAggregatingQuery();
@ -1564,6 +1585,9 @@ ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendOrderBy(Expr
getRootActions(select_query->orderBy(), only_types, step.actions()->dag);
for (const auto & result_column : step.getResultColumns())
validateOrderByKeyType(result_column.type);
bool with_fill = false;
for (auto & child : select_query->orderBy()->children)
@ -1643,6 +1667,24 @@ ActionsAndProjectInputsFlagPtr SelectQueryExpressionAnalyzer::appendOrderBy(Expr
return actions;
}
void SelectQueryExpressionAnalyzer::validateOrderByKeyType(const DataTypePtr & key_type) const
{
if (getContext()->getSettingsRef().allow_suspicious_types_in_order_by)
return;
auto check = [](const IDataType & type)
{
if (isDynamic(type) || isVariant(type))
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Data types Variant/Dynamic are not allowed in ORDER BY keys, because it can lead to unexpected results. "
"Set setting allow_suspicious_types_in_order_by = 1 in order to allow it");
};
check(*key_type);
key_type->forEachChild(check);
}
bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only_types)
{
const auto * select_query = getSelectQuery();

View File

@ -397,6 +397,7 @@ private:
ActionsAndProjectInputsFlagPtr appendPrewhere(ExpressionActionsChain & chain, bool only_types);
bool appendWhere(ExpressionActionsChain & chain, bool only_types);
bool appendGroupBy(ExpressionActionsChain & chain, bool only_types, bool optimize_aggregation_in_order, ManyExpressionActions &);
void validateGroupByKeyType(const DataTypePtr & key_type) const;
void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types);
void appendWindowFunctionsArguments(ExpressionActionsChain & chain, bool only_types);
@ -409,6 +410,7 @@ private:
bool appendHaving(ExpressionActionsChain & chain, bool only_types);
/// appendSelect
ActionsAndProjectInputsFlagPtr appendOrderBy(ExpressionActionsChain & chain, bool only_types, bool optimize_read_in_order, ManyExpressionActions &);
void validateOrderByKeyType(const DataTypePtr & key_type) const;
bool appendLimitBy(ExpressionActionsChain & chain, bool only_types);
/// appendProjectResult
};

View File

@ -151,6 +151,15 @@ KeyDescription KeyDescription::getSortingKeyFromAST(
throw Exception(ErrorCodes::DATA_TYPE_CANNOT_BE_USED_IN_KEY,
"Column {} with type {} is not allowed in key expression, it's not comparable",
backQuote(result.sample_block.getByPosition(i).name), result.data_types.back()->getName());
auto check = [&](const IDataType & type)
{
if (isDynamic(type) || isVariant(type))
throw Exception(ErrorCodes::DATA_TYPE_CANNOT_BE_USED_IN_KEY, "Column with type Variant/Dynamic is not allowed in key expression");
};
check(*result.data_types.back());
result.data_types.back()->forEachChild(check);
}
return result;

View File

@ -0,0 +1,184 @@
0
1
2
3
4
0
1
2
3
4
0
1
2
3
4
0
1
2
3
4
0
1
4
3
2
0
1
4
3
2
[4]
[3]
[2]
[0]
[1]
{'str':0}
{'str':1}
{'str':4}
{'str':3}
{'str':2}
0
1
4
3
2
\N
0
1
2
3
4
0
1
2
3
4
0
1
2
3
4
0
1
2
3
4
0
1
4
3
2
0
1
4
3
2
[4]
[3]
[2]
[0]
[1]
{'str':0}
{'str':1}
{'str':4}
{'str':3}
{'str':2}
\N
0
1
4
3
2
0
1
2
3
4
0
1
2
3
4
0
1
2
3
4
0
1
2
3
4
0
1
2
3
4
0
1
2
3
4
[4]
[0]
[1]
[2]
[3]
{'str':0}
{'str':1}
{'str':2}
{'str':3}
{'str':4}
0
1
2
3
4
\N
0
1
2
3
4
0
1
2
3
4
0
1
2
3
4
0
1
2
3
4
0
1
2
3
4
0
1
2
3
4
[4]
[0]
[1]
[2]
[3]
{'str':0}
{'str':1}
{'str':2}
{'str':3}
{'str':4}
0
1
2
3
4
\N

View File

@ -0,0 +1,154 @@
set allow_experimental_variant_type=1;
set allow_experimental_dynamic_type=1;
drop table if exists test;
create table test (d Dynamic) engine=MergeTree order by d; -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Dynamic) engine=MergeTree order by tuple(d); -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Dynamic) engine=MergeTree order by array(d); -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Dynamic) engine=MergeTree order by map('str', d); -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Dynamic) engine=MergeTree order by tuple() primary key d; -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Dynamic) engine=MergeTree order by tuple() partition by d; -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Dynamic) engine=MergeTree order by tuple() partition by tuple(d); -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Dynamic) engine=MergeTree order by tuple() partition by array(d); -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Dynamic) engine=MergeTree order by tuple() partition by map('str', d); -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Variant(UInt64)) engine=MergeTree order by d; -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Variant(UInt64)) engine=MergeTree order by tuple(d); -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Variant(UInt64)) engine=MergeTree order by array(d); -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Variant(UInt64)) engine=MergeTree order by map('str', d); -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Variant(UInt64)) engine=MergeTree order by tuple() primary key d; -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Variant(UInt64)) engine=MergeTree order by tuple() partition by d; -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Variant(UInt64)) engine=MergeTree order by tuple() partition by tuple(d); -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Variant(UInt64)) engine=MergeTree order by tuple() partition by array(d); -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Variant(UInt64)) engine=MergeTree order by tuple() partition by map('str', d); -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY}
create table test (d Dynamic) engine=Memory;
insert into test select * from numbers(5);
set allow_experimental_analyzer=1;
set allow_suspicious_types_in_group_by=0;
set allow_suspicious_types_in_order_by=0;
select * from test order by d; -- {serverError ILLEGAL_COLUMN}
select * from test order by tuple(d); -- {serverError ILLEGAL_COLUMN}
select * from test order by array(d); -- {serverError ILLEGAL_COLUMN}
select * from test order by map('str', d); -- {serverError ILLEGAL_COLUMN}
select * from test group by d; -- {serverError ILLEGAL_COLUMN}
select * from test group by tuple(d); -- {serverError ILLEGAL_COLUMN}
select array(d) from test group by array(d); -- {serverError ILLEGAL_COLUMN}
select map('str', d) from test group by map('str', d); -- {serverError ILLEGAL_COLUMN}
select * from test group by grouping sets ((d), ('str')); -- {serverError ILLEGAL_COLUMN}
set allow_suspicious_types_in_group_by=1;
set allow_suspicious_types_in_order_by=1;
select * from test order by d;
select * from test order by tuple(d);
select * from test order by array(d);
select * from test order by map('str', d);
select * from test group by d;
select * from test group by tuple(d);
select array(d) from test group by array(d);
select map('str', d) from test group by map('str', d);
select * from test group by grouping sets ((d), ('str'));
set allow_experimental_analyzer=0;
set allow_suspicious_types_in_group_by=0;
set allow_suspicious_types_in_order_by=0;
select * from test order by d; -- {serverError ILLEGAL_COLUMN}
select * from test order by tuple(d); -- {serverError ILLEGAL_COLUMN}
select * from test order by array(d); -- {serverError ILLEGAL_COLUMN}
select * from test order by map('str', d); -- {serverError ILLEGAL_COLUMN}
select * from test group by d; -- {serverError ILLEGAL_COLUMN}
select * from test group by tuple(d); -- {serverError ILLEGAL_COLUMN}
select array(d) from test group by array(d); -- {serverError ILLEGAL_COLUMN}
select map('str', d) from test group by map('str', d); -- {serverError ILLEGAL_COLUMN}
select * from test group by grouping sets ((d), ('str')); -- {serverError ILLEGAL_COLUMN}
set allow_suspicious_types_in_group_by=1;
set allow_suspicious_types_in_order_by=1;
select * from test order by d;
select * from test order by tuple(d);
select * from test order by array(d);
select * from test order by map('str', d);
select * from test group by d;
select * from test group by tuple(d);
select array(d) from test group by array(d);
select map('str', d) from test group by map('str', d);
select * from test group by grouping sets ((d), ('str'));
drop table test;
create table test (d Variant(UInt64)) engine=Memory;
insert into test select * from numbers(5);
set allow_experimental_analyzer=1;
set allow_suspicious_types_in_group_by=0;
set allow_suspicious_types_in_order_by=0;
select * from test order by d; -- {serverError ILLEGAL_COLUMN}
select * from test order by tuple(d); -- {serverError ILLEGAL_COLUMN}
select * from test order by array(d); -- {serverError ILLEGAL_COLUMN}
select * from test order by map('str', d); -- {serverError ILLEGAL_COLUMN}
select * from test group by d; -- {serverError ILLEGAL_COLUMN}
select * from test group by tuple(d); -- {serverError ILLEGAL_COLUMN}
select array(d) from test group by array(d); -- {serverError ILLEGAL_COLUMN}
select map('str', d) from test group by map('str', d); -- {serverError ILLEGAL_COLUMN}
select * from test group by grouping sets ((d), ('str')); -- {serverError ILLEGAL_COLUMN}
set allow_suspicious_types_in_group_by=1;
set allow_suspicious_types_in_order_by=1;
select * from test order by d;
select * from test order by tuple(d);
select * from test order by array(d);
select * from test order by map('str', d);
select * from test group by d;
select * from test group by tuple(d);
select array(d) from test group by array(d);
select map('str', d) from test group by map('str', d);
select * from test group by grouping sets ((d), ('str'));
set allow_experimental_analyzer=0;
set allow_suspicious_types_in_group_by=0;
set allow_suspicious_types_in_order_by=0;
select * from test order by d; -- {serverError ILLEGAL_COLUMN}
select * from test order by tuple(d); -- {serverError ILLEGAL_COLUMN}
select * from test order by array(d); -- {serverError ILLEGAL_COLUMN}
select * from test order by map('str', d); -- {serverError ILLEGAL_COLUMN}
select * from test group by d; -- {serverError ILLEGAL_COLUMN}
select * from test group by tuple(d); -- {serverError ILLEGAL_COLUMN}
select array(d) from test group by array(d); -- {serverError ILLEGAL_COLUMN}
select map('str', d) from test group by map('str', d); -- {serverError ILLEGAL_COLUMN}
select * from test group by grouping sets ((d), ('str')); -- {serverError ILLEGAL_COLUMN}
set allow_suspicious_types_in_group_by=1;
set allow_suspicious_types_in_order_by=1;
select * from test order by d;
select * from test order by tuple(d);
select * from test order by array(d);
select * from test order by map('str', d);
select * from test group by d;
select * from test group by tuple(d);
select array(d) from test group by array(d);
select map('str', d) from test group by map('str', d);
select * from test group by grouping sets ((d), ('str'));
drop table test;