Skip to content

Commit

Permalink
Forbid SimpleAggregateFunction in ORDER BY of MergeTree tables
Browse files Browse the repository at this point in the history
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>
  • Loading branch information
azat committed Mar 19, 2024
1 parent 7348760 commit 8b77f4e
Show file tree
Hide file tree
Showing 6 changed files with 38 additions and 4 deletions.
1 change: 1 addition & 0 deletions src/Core/Settings.h
Expand Up @@ -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) \
Expand Down
1 change: 1 addition & 0 deletions src/Core/SettingsChangesHistory.h
Expand Up @@ -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"},
Expand Down
24 changes: 21 additions & 3 deletions src/Storages/MergeTree/registerStorageMergeTree.cpp
Expand Up @@ -13,6 +13,7 @@
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSetQuery.h>

#include <DataTypes/DataTypeCustomSimpleAggregateFunction.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>

#include <Interpreters/Context.h>
Expand All @@ -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;
}


Expand Down Expand Up @@ -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)
{
Expand Down Expand Up @@ -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");
Expand Down Expand Up @@ -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. "
Expand Down Expand Up @@ -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"));
}
Expand All @@ -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)
Expand All @@ -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)
{
Expand Down Expand Up @@ -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);
Expand Down
2 changes: 1 addition & 1 deletion tests/queries/0_stateless/01410_nullable_key_and_index.sql
Expand Up @@ -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.
Empty 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);

0 comments on commit 8b77f4e

Please sign in to comment.