Forbid SimpleAggregateFunction in ORDER BY of MergeTree tables

Like AggregateFunction is forbidden, but they are forbidden because they
are not comparable.

New setting (allow_suspicious_primary_key) had been added for backard
compatiblity (turned OFF by default).

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
This commit is contained in:
Azat Khuzhin 2024-03-14 19:22:02 +01:00
parent 99a81c848e
commit 11d0b0a9fa
6 changed files with 38 additions and 4 deletions

View File

@ -161,6 +161,7 @@ class IColumn;
M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \
M(Bool, allow_suspicious_ttl_expressions, false, "Reject TTL expressions that don't depend on any of table's columns. It indicates a user error most of the time.", 0) \
M(Bool, allow_suspicious_variant_types, false, "In CREATE TABLE statement allows specifying Variant type with similar variant types (for example, with different numeric or date types). Enabling this setting may introduce some ambiguity when working with values with similar types.", 0) \
M(Bool, allow_suspicious_primary_key, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)", 0) \
M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \
M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \
M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code.", 0) \

View File

@ -95,6 +95,7 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication is dependent materialized view cannot work together with async inserts."},
{"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"},
{"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."},
{"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"},
{"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"},
{"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"},
{"keeper_max_retries", 10, 10, "Max retries for general keeper operations"},

View File

@ -13,6 +13,7 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSetQuery.h>
#include <DataTypes/DataTypeCustomSimpleAggregateFunction.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <Interpreters/Context.h>
@ -30,6 +31,7 @@ namespace ErrorCodes
extern const int UNKNOWN_STORAGE;
extern const int NO_REPLICA_NAME_GIVEN;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
extern const int DATA_TYPE_CANNOT_BE_USED_IN_KEY;
}
@ -110,6 +112,16 @@ static ColumnsDescription getColumnsDescriptionFromZookeeper(const String & raw_
return ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_path) / "columns", &columns_stat));
}
static void verifySortingKey(const KeyDescription & sorting_key)
{
/// Aggregate functions already forbidden, but SimpleAggregateFunction are not
for (const auto & data_type : sorting_key.data_types)
{
if (dynamic_cast<const DataTypeCustomSimpleAggregateFunction *>(data_type->getCustomName()))
throw Exception(ErrorCodes::DATA_TYPE_CANNOT_BE_USED_IN_KEY, "Column with type {} is not allowed in key expression", data_type->getCustomName()->getName());
}
}
static StoragePtr create(const StorageFactory::Arguments & args)
{
@ -148,6 +160,8 @@ static StoragePtr create(const StorageFactory::Arguments & args)
|| (args.query.columns_list->indices && !args.query.columns_list->indices->children.empty())
|| (args.query.columns_list->projections && !args.query.columns_list->projections->children.empty());
const Settings & local_settings = args.getLocalContext()->getSettingsRef();
String name_part = args.engine_name.substr(0, args.engine_name.size() - strlen("MergeTree"));
bool replicated = startsWith(name_part, "Replicated");
@ -293,7 +307,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
arg_idx, e.message(), verbose_help_message);
}
}
else if (args.mode <= LoadingStrictnessLevel::CREATE && !args.getLocalContext()->getSettingsRef().allow_deprecated_syntax_for_merge_tree)
else if (args.mode <= LoadingStrictnessLevel::CREATE && !local_settings.allow_deprecated_syntax_for_merge_tree)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "This syntax for *MergeTree engine is deprecated. "
"Use extended storage definition syntax with ORDER BY/PRIMARY KEY clause. "
@ -532,7 +546,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
if (!args.storage_def->order_by)
{
if (args.getLocalContext()->getSettingsRef().create_table_empty_primary_key_by_default)
if (local_settings.create_table_empty_primary_key_by_default)
{
args.storage_def->set(args.storage_def->order_by, makeASTFunction("tuple"));
}
@ -553,6 +567,8 @@ static StoragePtr create(const StorageFactory::Arguments & args)
/// column if sorting key will be changed.
metadata.sorting_key = KeyDescription::getSortingKeyFromAST(
args.storage_def->order_by->ptr(), metadata.columns, context, merging_param_key_arg);
if (!local_settings.allow_suspicious_primary_key)
verifySortingKey(metadata.sorting_key);
/// If primary key explicitly defined, than get it from AST
if (args.storage_def->primary_key)
@ -577,7 +593,7 @@ static StoragePtr create(const StorageFactory::Arguments & args)
if (args.storage_def->sample_by)
metadata.sampling_key = KeyDescription::getKeyFromAST(args.storage_def->sample_by->ptr(), metadata.columns, context);
bool allow_suspicious_ttl = LoadingStrictnessLevel::SECONDARY_CREATE <= args.mode || args.getLocalContext()->getSettingsRef().allow_suspicious_ttl_expressions;
bool allow_suspicious_ttl = LoadingStrictnessLevel::SECONDARY_CREATE <= args.mode || local_settings.allow_suspicious_ttl_expressions;
if (args.storage_def->ttl_table)
{
@ -665,6 +681,8 @@ static StoragePtr create(const StorageFactory::Arguments & args)
/// column if sorting key will be changed.
metadata.sorting_key
= KeyDescription::getSortingKeyFromAST(engine_args[arg_num], metadata.columns, context, merging_param_key_arg);
if (!local_settings.allow_suspicious_primary_key)
verifySortingKey(metadata.sorting_key);
/// In old syntax primary_key always equals to sorting key.
metadata.primary_key = KeyDescription::getKeyFromAST(engine_args[arg_num], metadata.columns, context);

View File

@ -73,5 +73,5 @@ CREATE TABLE invalid_lc_null (id LowCardinality(Nullable(String))) ENGINE = Merg
CREATE TABLE invalid_array_null (id Array(Nullable(String))) ENGINE = MergeTree ORDER BY id; -- { serverError 44 }
CREATE TABLE invalid_tuple_null (id Tuple(Nullable(String), UInt8)) ENGINE = MergeTree ORDER BY id; -- { serverError 44 }
CREATE TABLE invalid_map_null (id Map(UInt8, Nullable(String))) ENGINE = MergeTree ORDER BY id; -- { serverError 44 }
CREATE TABLE invalid_simple_agg_state_null (id SimpleAggregateFunction(sum, Nullable(UInt64))) ENGINE = MergeTree ORDER BY id; -- { serverError 44 }
CREATE TABLE invalid_simple_agg_state_null (id SimpleAggregateFunction(sum, Nullable(UInt64))) ENGINE = MergeTree ORDER BY id; -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY }
-- AggregateFunctions are not comparable and cannot be used in key expressions. No need to test it.

View File

@ -0,0 +1,14 @@
set allow_suspicious_primary_key = 0;
create table data (key Int, value AggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() order by (key, value); -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY }
create table data (key Int, value SimpleAggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() order by (key, value); -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY }
create table data (key Int, value AggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() primary key value; -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY }
create table data (key Int, value SimpleAggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() primary key value; -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY }
create table data (key Int, value AggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() primary key value order by (value, key); -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY }
create table data (key Int, value SimpleAggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() primary key value order by (value, key); -- { serverError DATA_TYPE_CANNOT_BE_USED_IN_KEY }
set allow_suspicious_primary_key = 1;
create table data (key Int, value SimpleAggregateFunction(sum, UInt64)) engine=AggregatingMergeTree() primary key value order by (value, key);