From e0d3213481769299881044512d78d23551c2aa8a Mon Sep 17 00:00:00 2001 From: Eduard Karacharov Date: Sun, 16 Jun 2024 20:43:26 +0300 Subject: [PATCH 1/2] support set transformation in partition pruning --- src/Storages/MergeTree/KeyCondition.cpp | 202 +++++++++++++- src/Storages/MergeTree/KeyCondition.h | 14 +- ...et_transformed_partition_pruning.reference | 50 ++++ ...3173_set_transformed_partition_pruning.sql | 258 ++++++++++++++++++ 4 files changed, 521 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/03173_set_transformed_partition_pruning.reference create mode 100644 tests/queries/0_stateless/03173_set_transformed_partition_pruning.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 7e4b1db4c89..85eca4644e7 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1186,15 +1186,62 @@ bool KeyCondition::canConstantBeWrappedByFunctions( }); } +/// Sequentially applies functions to set column, returns `true` +/// if all function arguments are compatible with functions +/// signatures, and none of the functions produce `NULL` output. +/// +/// After functions chain execution, fills result set column and +/// its type. +bool applyFunctionChainToSetValues( + const ColumnPtr & set_column, + const DataTypePtr & set_data_type, + const std::vector & functions, + ColumnPtr & out_set_column, + DataTypePtr & out_set_type) +{ + auto result_column = set_column->convertToFullColumnIfLowCardinality(); + auto result_type = removeLowCardinality(set_data_type); + + for (const auto & func : functions) + { + if (func->getArgumentTypes().empty()) + return false; + + auto argument_type = func->getArgumentTypes()[0]; + if (!canBeSafelyCasted(result_type, argument_type)) + return false; + + result_column = castColumnAccurate({result_column, result_type, ""}, argument_type); + result_column = func->execute({{result_column, argument_type, ""}}, func->getResultType(), result_column->size()); + if (result_column->isNullable()) + { + const auto & result_column_nullable = assert_cast(*result_column); + const auto & null_map_data = result_column_nullable.getNullMapData(); + for (char8_t i : null_map_data) + { + if (i != 0) + return false; + } + } + result_type = func->getResultType(); + } + out_set_column = result_column; + out_set_type = result_type; + + return true; +} + bool KeyCondition::tryPrepareSetIndex( const RPNBuilderFunctionTreeNode & func, RPNElement & out, - size_t & out_key_column_num) + size_t & out_key_column_num, + bool & is_constant_transformed) { const auto & left_arg = func.getArgumentAt(0); out_key_column_num = 0; std::vector indexes_mapping; + std::vector set_transforming_chains; DataTypes data_types; auto get_key_tuple_position_mapping = [&](const RPNBuilderTreeNode & node, size_t tuple_index) @@ -1203,6 +1250,7 @@ bool KeyCondition::tryPrepareSetIndex( index_mapping.tuple_index = tuple_index; DataTypePtr data_type; std::optional key_space_filling_curve_argument_pos; + MonotonicFunctionsChain set_transforming_chain; if (isKeyPossiblyWrappedByMonotonicFunctions( node, index_mapping.key_index, key_space_filling_curve_argument_pos, data_type, index_mapping.functions) && !key_space_filling_curve_argument_pos) /// We don't support the analysis of space-filling curves and IN set. @@ -1210,6 +1258,14 @@ bool KeyCondition::tryPrepareSetIndex( indexes_mapping.push_back(index_mapping); data_types.push_back(data_type); out_key_column_num = std::max(out_key_column_num, index_mapping.key_index); + set_transforming_chains.push_back(set_transforming_chain); + } + else if (single_point && canSetValuesBeWrappedByFunctions(node, index_mapping.key_index, data_type, set_transforming_chain)) + { + indexes_mapping.push_back(index_mapping); + data_types.push_back(data_type); + out_key_column_num = std::max(out_key_column_num, index_mapping.key_index); + set_transforming_chains.push_back(set_transforming_chain); } }; @@ -1275,6 +1331,18 @@ bool KeyCondition::tryPrepareSetIndex( auto set_element_type = set_types[set_element_index]; auto set_column = set_columns[set_element_index]; + if (!set_transforming_chains[indexes_mapping_index].empty()) + { + ColumnPtr transformed_set_column; + DataTypePtr transformed_set_type; + if (!applyFunctionChainToSetValues(set_column, set_element_type, set_transforming_chains[indexes_mapping_index], transformed_set_column, transformed_set_type)) + return false; + + set_column = transformed_set_column; + set_element_type = transformed_set_type; + is_constant_transformed = true; + } + if (canBeSafelyCasted(set_element_type, key_column_type)) { set_columns[set_element_index] = castColumn({set_column, set_element_type, {}}, key_column_type); @@ -1571,6 +1639,136 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( return false; } +bool KeyCondition::canSetValuesBeWrappedByFunctions( + const RPNBuilderTreeNode & node, + size_t & out_key_column_num, + DataTypePtr & out_key_res_column_type, + MonotonicFunctionsChain & out_functions_chain) +{ + // Checking if column name matches any of key subexpressions + String expr_name = node.getColumnName(); + + if (array_joined_column_names.contains(expr_name)) + return false; + + if (!key_subexpr_names.contains(expr_name)) + { + expr_name = node.getColumnNameWithModuloLegacy(); + + if (!key_subexpr_names.contains(expr_name)) + return false; + } + + // If match has been found, need to identify key column, and + // sequence of function nodes, which forms key column + for (const auto & key_expr_node : key_expr->getNodes()) + { + auto it = key_columns.find(key_expr_node.result_name); + if (it != key_columns.end()) + { + std::vector chain; + + const auto * cur_node = &key_expr_node; + bool is_valid_chain = true; + + while (is_valid_chain) + { + if (cur_node->result_name == expr_name) + break; + + if (cur_node->type == ActionsDAG::ActionType::FUNCTION && cur_node->children.size() <= 2) + { + chain.push_back(cur_node); + is_valid_chain = cur_node->function_base->isDeterministic(); + + const ActionsDAG::Node * next_node = nullptr; + for (const auto * arg : cur_node->children) + { + if (arg->column && isColumnConst(*arg->column)) + continue; + + if (next_node) + is_valid_chain = false; + + next_node = arg; + } + + if (!next_node) + is_valid_chain = false; + + cur_node = next_node; + } + else if (cur_node->type == ActionsDAG::ActionType::ALIAS) + cur_node = cur_node->children.front(); + else + is_valid_chain = false; + } + + // If key column has been found, and it consists of deterministic function, + // in order to make them applicable to constants from query predicate, + // reversing the sequence of functions, and binding constant arguments + if (is_valid_chain) + { + // Removing low cardinality from `node` output type to avoid + // passing it through functions signatures + auto nonconst_input_type = removeLowCardinality(node.getDAGNode()->result_type); + + for (auto iter = chain.rbegin(); iter != chain.rend(); ++iter) + { + const auto * function_node = *iter; + auto function = function_node->function_base; + auto func_builder = FunctionFactory::instance().tryGet(function->getName(), node.getTreeContext().getQueryContext()); + if (!func_builder) + return false; + ColumnsWithTypeAndName arguments; + ColumnWithTypeAndName const_arg; + FunctionWithOptionalConstArg::Kind kind = FunctionWithOptionalConstArg::Kind::NO_CONST; + if (function_node->children.size() == 2) + { + const auto * left = function_node->children[0]; + const auto * right = function_node->children[1]; + if (left->column && isColumnConst(*left->column)) + { + const_arg = {left->result_type->createColumnConst(0, (*left->column)[0]), left->result_type, ""}; + arguments.push_back(const_arg); + arguments.push_back({nullptr, nonconst_input_type, ""}); + kind = FunctionWithOptionalConstArg::Kind::LEFT_CONST; + } + else if (right->column && isColumnConst(*right->column)) + { + const_arg = {right->result_type->createColumnConst(0, (*right->column)[0]), right->result_type, ""}; + arguments.push_back({nullptr, nonconst_input_type, ""}); + arguments.push_back(const_arg); + kind = FunctionWithOptionalConstArg::Kind::RIGHT_CONST; + } + + /// If constant arg of binary operator is NULL, there will be no monotonicity. + if (const_arg.column->isNullAt(0)) + return false; + } + else + arguments.push_back({nullptr, nonconst_input_type, ""}); + auto func = func_builder->build(arguments); + + if (kind == FunctionWithOptionalConstArg::Kind::NO_CONST) + out_functions_chain.push_back(func); + else + out_functions_chain.push_back(std::make_shared(func, const_arg, kind)); + + nonconst_input_type = func->getResultType(); + } + + const auto & sample_block = key_expr->getSampleBlock(); + out_key_column_num = it->second; + out_key_res_column_type = sample_block.getByName(it->first).type; + + return true; + } + } + } + + return false; +} static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const String & node_column_name) { @@ -1649,7 +1847,7 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme if (functionIsInOrGlobalInOperator(func_name)) { - if (tryPrepareSetIndex(func, out, key_column_num)) + if (tryPrepareSetIndex(func, out, key_column_num, is_constant_transformed)) { key_arg_pos = 0; is_set_const = true; diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 6e5956706aa..11dd1c85c9d 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -276,13 +276,25 @@ private: Field & out_value, DataTypePtr & out_type); + /// Checks if node is a subexpression of any of key columns expressions, + /// wrapped by deterministic functions, and if so, returns `true`, and + /// specifies key column position / type. Besides that it produces the + /// chain of functions which should be executed on set, to transform it + /// into key column values. + bool canSetValuesBeWrappedByFunctions( + const RPNBuilderTreeNode & node, + size_t & out_key_column_num, + DataTypePtr & out_key_res_column_type, + MonotonicFunctionsChain & out_functions_chain); + /// If it's possible to make an RPNElement /// that will filter values (possibly tuples) by the content of 'prepared_set', /// do it and return true. bool tryPrepareSetIndex( const RPNBuilderFunctionTreeNode & func, RPNElement & out, - size_t & out_key_column_num); + size_t & out_key_column_num, + bool & is_constant_transformed); /// Checks that the index can not be used. /// diff --git a/tests/queries/0_stateless/03173_set_transformed_partition_pruning.reference b/tests/queries/0_stateless/03173_set_transformed_partition_pruning.reference new file mode 100644 index 00000000000..3a6727b70e8 --- /dev/null +++ b/tests/queries/0_stateless/03173_set_transformed_partition_pruning.reference @@ -0,0 +1,50 @@ +-- Single partition by function +0 +2 +-- Nested partition by function +1 +2 +1 +1 +-- Nested partition by function, LowCardinality +1 +2 +1 +1 +-- Nested partition by function, Nullable +1 +2 +1 +1 +-- Nested partition by function, LowCardinality + Nullable +1 +2 +1 +1 +-- Non-safe cast +2 +2 +-- Multiple partition columns +1 +1 +1 +2 +-- LowCardinality set +1 +1 +-- Nullable set +1 +1 +-- LowCardinality + Nullable set +1 +1 +-- Not failing with date parsing functions +1 +0 +-- Pruning + not failing with nested date parsing functions +1 +2 +0 +-- Empty transform functions +2 +1 diff --git a/tests/queries/0_stateless/03173_set_transformed_partition_pruning.sql b/tests/queries/0_stateless/03173_set_transformed_partition_pruning.sql new file mode 100644 index 00000000000..8ffabacaa8c --- /dev/null +++ b/tests/queries/0_stateless/03173_set_transformed_partition_pruning.sql @@ -0,0 +1,258 @@ +SELECT '-- Single partition by function'; + +DROP TABLE IF EXISTS 03173_single_function; +CREATE TABLE 03173_single_function ( + dt Date, +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY toMonth(dt); + +INSERT INTO 03173_single_function +SELECT toDate('2000-01-01') + 10 * number FROM numbers(50) +UNION ALL +SELECT toDate('2100-01-01') + 10 * number FROM numbers(50); +OPTIMIZE TABLE 03173_single_function FINAL; + +SELECT count() FROM 03173_single_function WHERE dt IN ('2024-01-20', '2024-05-25') SETTINGS log_comment='03173_single_function'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_single_function'; + +DROP TABLE IF EXISTS 03173_single_function; + +SELECT '-- Nested partition by function'; + +DROP TABLE IF EXISTS 03173_nested_function; +CREATE TABLE 03173_nested_function( + id Int32, +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY xxHash32(id) % 3; + +INSERT INTO 03173_nested_function SELECT number FROM numbers(100); +OPTIMIZE TABLE 03173_nested_function FINAL; + +SELECT count() FROM 03173_nested_function WHERE id IN (10) SETTINGS log_comment='03173_nested_function'; +SELECT count() FROM 03173_nested_function WHERE xxHash32(id) IN (2158931063, 1449383981) SETTINGS log_comment='03173_nested_function_subexpr'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function'; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function_subexpr'; + +DROP TABLE IF EXISTS 03173_nested_function; + +SELECT '-- Nested partition by function, LowCardinality'; + +SET allow_suspicious_low_cardinality_types = 1; + +DROP TABLE IF EXISTS 03173_nested_function_lc; +CREATE TABLE 03173_nested_function_lc( + id LowCardinality(Int32), +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY xxHash32(id) % 3; + +INSERT INTO 03173_nested_function_lc SELECT number FROM numbers(100); +OPTIMIZE TABLE 03173_nested_function_lc FINAL; + +SELECT count() FROM 03173_nested_function_lc WHERE id IN (10) SETTINGS log_comment='03173_nested_function_lc'; +SELECT count() FROM 03173_nested_function_lc WHERE xxHash32(id) IN (2158931063, 1449383981) SETTINGS log_comment='03173_nested_function_subexpr_lc'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function_lc'; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function_subexpr_lc'; + +DROP TABLE IF EXISTS 03173_nested_function_lc; + +SELECT '-- Nested partition by function, Nullable'; + +DROP TABLE IF EXISTS 03173_nested_function_null; +CREATE TABLE 03173_nested_function_null( + id Nullable(Int32), +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY xxHash32(id) % 3 +SETTINGS allow_nullable_key=1; + +INSERT INTO 03173_nested_function_null SELECT number FROM numbers(100); +OPTIMIZE TABLE 03173_nested_function_null FINAL; + +SELECT count() FROM 03173_nested_function_null WHERE id IN (10) SETTINGS log_comment='03173_nested_function_null'; +SELECT count() FROM 03173_nested_function_null WHERE xxHash32(id) IN (2158931063, 1449383981) SETTINGS log_comment='03173_nested_function_subexpr_null'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function_null'; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function_subexpr_null'; + +DROP TABLE IF EXISTS 03173_nested_function_null; + +SELECT '-- Nested partition by function, LowCardinality + Nullable'; + +DROP TABLE IF EXISTS 03173_nested_function_lc_null; + +SET allow_suspicious_low_cardinality_types = 1; +CREATE TABLE 03173_nested_function_lc_null( + id LowCardinality(Nullable(Int32)), +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY xxHash32(id) % 3 +SETTINGS allow_nullable_key=1; + +INSERT INTO 03173_nested_function_lc_null SELECT number FROM numbers(100); +OPTIMIZE TABLE 03173_nested_function_lc_null FINAL; + +SELECT count() FROM 03173_nested_function_lc_null WHERE id IN (10) SETTINGS log_comment='03173_nested_function_lc_null'; +SELECT count() FROM 03173_nested_function_lc_null WHERE xxHash32(id) IN (2158931063, 1449383981) SETTINGS log_comment='03173_nested_function_subexpr_lc_null'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function_lc_null'; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_function_subexpr_lc_null'; + +DROP TABLE IF EXISTS 03173_nested_function_lc_null; + +SELECT '-- Non-safe cast'; + +DROP TABLE IF EXISTS 03173_nonsafe_cast; +CREATE TABLE 03173_nonsafe_cast( + id Int64, +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY xxHash32(id) % 3; + +INSERT INTO 03173_nonsafe_cast SELECT number FROM numbers(100); +OPTIMIZE TABLE 03173_nonsafe_cast FINAL; + +SELECT count() FROM 03173_nonsafe_cast WHERE id IN (SELECT '50' UNION ALL SELECT '99') SETTINGS log_comment='03173_nonsafe_cast'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nonsafe_cast'; + +DROP TABLE IF EXISTS 03173_nonsafe_cast; + +SELECT '-- Multiple partition columns'; + +DROP TABLE IF EXISTS 03173_multiple_partition_cols; +CREATE TABLE 03173_multiple_partition_cols ( + key1 Int32, + key2 Int32 +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY (intDiv(key1, 50), xxHash32(key2) % 3); + +INSERT INTO 03173_multiple_partition_cols SELECT number, number FROM numbers(100); +OPTIMIZE TABLE 03173_multiple_partition_cols FINAL; + +SELECT count() FROM 03173_multiple_partition_cols WHERE key2 IN (4) SETTINGS log_comment='03173_multiple_columns'; +SELECT count() FROM 03173_multiple_partition_cols WHERE xxHash32(key2) IN (4251411170) SETTINGS log_comment='03173_multiple_columns_subexpr'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_multiple_columns'; +-- Due to xxHash32() in WHERE condition, MinMax is unable to eliminate any parts, +-- so partition pruning leave two parts (for key1 // 50 = 0 and key1 // 50 = 1) +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_multiple_columns_subexpr'; + +-- Preparing base table for filtering by LowCardinality/Nullable sets +DROP TABLE IF EXISTS 03173_base_data_source; +CREATE TABLE 03173_base_data_source( + id Int32, +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY xxHash32(id) % 3; + +INSERT INTO 03173_base_data_source SELECT number FROM numbers(100); +OPTIMIZE TABLE 03173_base_data_source FINAL; + +SELECT '-- LowCardinality set'; + +SET allow_suspicious_low_cardinality_types = 1; +DROP TABLE IF EXISTS 03173_low_cardinality_set; +CREATE TABLE 03173_low_cardinality_set (id LowCardinality(Int32)) ENGINE=Memory AS SELECT 10; + +SELECT count() FROM 03173_base_data_source WHERE id IN (SELECT id FROM 03173_low_cardinality_set) SETTINGS log_comment='03173_low_cardinality_set'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_low_cardinality_set'; + +DROP TABLE IF EXISTS 03173_low_cardinality_set; + +SELECT '-- Nullable set'; + +DROP TABLE IF EXISTS 03173_nullable_set; +CREATE TABLE 03173_nullable_set (id Nullable(Int32)) ENGINE=Memory AS SELECT 10; + +SELECT count() FROM 03173_base_data_source WHERE id IN (SELECT id FROM 03173_nullable_set) SETTINGS log_comment='03173_nullable_set'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nullable_set'; + +DROP TABLE IF EXISTS 03173_nullable_set; + +SELECT '-- LowCardinality + Nullable set'; + +DROP TABLE IF EXISTS 03173_lc_nullable_set; +CREATE TABLE 03173_lc_nullable_set (id LowCardinality(Nullable(Int32))) ENGINE=Memory AS SELECT 10 UNION ALL SELECT NULL; + +SELECT count() FROM 03173_base_data_source WHERE id IN (SELECT id FROM 03173_lc_nullable_set) SETTINGS log_comment='03173_lc_nullable_set'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_lc_nullable_set'; + +DROP TABLE IF EXISTS 03173_lc_nullable_set; + +SELECT '-- Not failing with date parsing functions'; + +DROP TABLE IF EXISTS 03173_date_parsing; +CREATE TABLE 03173_date_parsing ( + id String +) +ENGINE=MergeTree +ORDER BY tuple() +PARTITION BY toDate(id); + +INSERT INTO 03173_date_parsing +SELECT toString(toDate('2023-04-01') + number) +FROM numbers(20); + +SELECT count() FROM 03173_date_parsing WHERE id IN ('2023-04-02', '2023-05-02'); +SELECT count() FROM 03173_date_parsing WHERE id IN ('not a date'); + +DROP TABLE IF EXISTS 03173_date_parsing; + +SELECT '-- Pruning + not failing with nested date parsing functions'; + +DROP TABLE IF EXISTS 03173_nested_date_parsing; +CREATE TABLE 03173_nested_date_parsing ( + id String +) +ENGINE=MergeTree +ORDER BY tuple() +PARTITION BY toMonth(toDate(id)); + +INSERT INTO 03173_nested_date_parsing +SELECT toString(toDate('2000-01-01') + 10 * number) FROM numbers(50) +UNION ALL +SELECT toString(toDate('2100-01-01') + 10 * number) FROM numbers(50); + +SELECT count() FROM 03173_nested_date_parsing WHERE id IN ('2000-01-21', '2023-05-02') SETTINGS log_comment='03173_nested_date_parsing'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_nested_date_parsing'; +SELECT count() FROM 03173_nested_date_parsing WHERE id IN ('not a date'); + +DROP TABLE IF EXISTS 03173_nested_date_parsing; + +SELECT '-- Empty transform functions'; + +DROP TABLE IF EXISTS 03173_empty_transform; +CREATE TABLE 03173_empty_transform( + id Int32, +) +ENGINE = MergeTree +ORDER BY tuple() +PARTITION BY xxHash32(id) % 3; + +INSERT INTO 03173_empty_transform SELECT number FROM numbers(6); +OPTIMIZE TABLE 03173_empty_transform FINAL; + +SELECT id FROM 03173_empty_transform WHERE xxHash32(id) % 3 IN (xxHash32(2::Int32) % 3) SETTINGS log_comment='03173_empty_transform'; +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['SelectedParts'] FROM system.query_log WHERE type = 'QueryFinish' AND current_database = currentDatabase() AND log_comment = '03173_empty_transform'; + +DROP TABLE IF EXISTS 03173_empty_transform; From b2306fc67acb992afbe25ef1a0ee9714ed19fbd0 Mon Sep 17 00:00:00 2001 From: Eduard Karacharov Date: Wed, 19 Jun 2024 09:32:29 +0300 Subject: [PATCH 2/2] process sets and scalars by same functions --- src/Storages/MergeTree/KeyCondition.cpp | 549 +++++++++++------------- src/Storages/MergeTree/KeyCondition.h | 6 +- 2 files changed, 249 insertions(+), 306 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 85eca4644e7..d38001a0feb 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -874,46 +874,6 @@ static Field applyFunctionForField( return (*col)[0]; } -/// The case when arguments may have types different than in the primary key. -static std::pair applyFunctionForFieldOfUnknownType( - const FunctionBasePtr & func, - const DataTypePtr & arg_type, - const Field & arg_value) -{ - ColumnsWithTypeAndName arguments{{ arg_type->createColumnConst(1, arg_value), arg_type, "x" }}; - DataTypePtr return_type = func->getResultType(); - - auto col = func->execute(arguments, return_type, 1); - - Field result = (*col)[0]; - - return {std::move(result), std::move(return_type)}; -} - - -/// Same as above but for binary operators -static std::pair applyBinaryFunctionForFieldOfUnknownType( - const FunctionOverloadResolverPtr & func, - const DataTypePtr & arg_type, - const Field & arg_value, - const DataTypePtr & arg_type2, - const Field & arg_value2) -{ - ColumnsWithTypeAndName arguments{ - {arg_type->createColumnConst(1, arg_value), arg_type, "x"}, {arg_type2->createColumnConst(1, arg_value2), arg_type2, "y"}}; - - FunctionBasePtr func_base = func->build(arguments); - - DataTypePtr return_type = func_base->getResultType(); - - auto col = func_base->execute(arguments, return_type, 1); - - Field result = (*col)[0]; - - return {std::move(result), std::move(return_type)}; -} - - static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field) { /// Fallback for fields without block reference. @@ -940,164 +900,92 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & return {field.columns, field.row_idx, result_idx}; } -/** When table's key has expression with these functions from a column, - * and when a column in a query is compared with a constant, such as: - * CREATE TABLE (x String) ORDER BY toDate(x) - * SELECT ... WHERE x LIKE 'Hello%' - * we want to apply the function to the constant for index analysis, - * but should modify it to pass on un-parsable values. - */ -static std::set date_time_parsing_functions = { - "toDate", - "toDate32", - "toDateTime", - "toDateTime64", - "parseDateTimeBestEffort", - "parseDateTimeBestEffortUS", - "parseDateTime32BestEffort", - "parseDateTime64BestEffort", - "parseDateTime", - "parseDateTimeInJodaSyntax", -}; - -/** The key functional expression constraint may be inferred from a plain column in the expression. - * For example, if the key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`, - * it can be assumed that if `toStartOfHour()` is monotonic on [now(), inf), the `toStartOfHour(Timestamp) >= toStartOfHour(now())` - * condition also holds, so the index may be used to select only parts satisfying this condition. - * - * To check the assumption, we'd need to assert that the inverse function to this transformation is also monotonic, however the - * inversion isn't exported (or even viable for not strictly monotonic functions such as `toStartOfHour()`). - * Instead, we can qualify only functions that do not transform the range (for example rounding), - * which while not strictly monotonic, are monotonic everywhere on the input range. - */ -bool KeyCondition::transformConstantWithValidFunctions( - ContextPtr context, - const String & expr_name, - size_t & out_key_column_num, - DataTypePtr & out_key_column_type, - Field & out_value, - DataTypePtr & out_type, - std::function always_monotonic) const +/// Sequentially applies functions to the column, returns `true` +/// if all function arguments are compatible with functions +/// signatures, and none of the functions produce `NULL` output. +/// +/// After functions chain execution, fills result column and its type. +bool applyFunctionChainToColumn( + const ColumnPtr & in_column, + const DataTypePtr & in_data_type, + const std::vector & functions, + ColumnPtr & out_column, + DataTypePtr & out_data_type) { - const auto & sample_block = key_expr->getSampleBlock(); + // Remove LowCardinality from input column, and convert it to regular one + auto result_column = in_column->convertToFullIfNeeded(); + auto result_type = removeLowCardinality(in_data_type); - for (const auto & node : key_expr->getNodes()) + // In case function sequence is empty, return full non-LowCardinality column + if (functions.empty()) { - auto it = key_columns.find(node.result_name); - if (it != key_columns.end()) - { - std::stack chain; - - const auto * cur_node = &node; - bool is_valid_chain = true; - - while (is_valid_chain) - { - if (cur_node->result_name == expr_name) - break; - - chain.push(cur_node); - - if (cur_node->type == ActionsDAG::ActionType::FUNCTION && cur_node->children.size() <= 2) - { - is_valid_chain = always_monotonic(*cur_node->function_base, *cur_node->result_type); - - const ActionsDAG::Node * next_node = nullptr; - for (const auto * arg : cur_node->children) - { - if (arg->column && isColumnConst(*arg->column)) - continue; - - if (next_node) - is_valid_chain = false; - - next_node = arg; - } - - if (!next_node) - is_valid_chain = false; - - cur_node = next_node; - } - else if (cur_node->type == ActionsDAG::ActionType::ALIAS) - cur_node = cur_node->children.front(); - else - is_valid_chain = false; - } - - if (is_valid_chain) - { - out_type = removeLowCardinality(out_type); - auto const_type = removeLowCardinality(cur_node->result_type); - auto const_column = out_type->createColumnConst(1, out_value); - auto const_value = (*castColumnAccurateOrNull({const_column, out_type, ""}, const_type))[0]; - - if (const_value.isNull()) - return false; - - while (!chain.empty()) - { - const auto * func = chain.top(); - chain.pop(); - - if (func->type != ActionsDAG::ActionType::FUNCTION) - continue; - - const auto & func_name = func->function_base->getName(); - auto func_base = func->function_base; - const auto & arg_types = func_base->getArgumentTypes(); - if (date_time_parsing_functions.contains(func_name) && !arg_types.empty() && isStringOrFixedString(arg_types[0])) - { - auto func_or_null = FunctionFactory::instance().get(func_name + "OrNull", context); - ColumnsWithTypeAndName arguments; - int i = 0; - for (const auto & type : func->function_base->getArgumentTypes()) - arguments.push_back({nullptr, type, fmt::format("_{}", i++)}); - - func_base = func_or_null->build(arguments); - } - - if (func->children.size() == 1) - { - std::tie(const_value, const_type) - = applyFunctionForFieldOfUnknownType(func_base, const_type, const_value); - } - else if (func->children.size() == 2) - { - const auto * left = func->children[0]; - const auto * right = func->children[1]; - if (left->column && isColumnConst(*left->column)) - { - auto left_arg_type = left->result_type; - auto left_arg_value = (*left->column)[0]; - std::tie(const_value, const_type) = applyBinaryFunctionForFieldOfUnknownType( - FunctionFactory::instance().get(func_base->getName(), context), - left_arg_type, left_arg_value, const_type, const_value); - } - else - { - auto right_arg_type = right->result_type; - auto right_arg_value = (*right->column)[0]; - std::tie(const_value, const_type) = applyBinaryFunctionForFieldOfUnknownType( - FunctionFactory::instance().get(func_base->getName(), context), - const_type, const_value, right_arg_type, right_arg_value); - } - } - - if (const_value.isNull()) - return false; - } - - out_key_column_num = it->second; - out_key_column_type = sample_block.getByName(it->first).type; - out_value = const_value; - out_type = const_type; - return true; - } - } + out_column = result_column; + out_data_type = result_type; + return true; } - return false; + // If first function arguments are empty, cannot transform input column + if (functions[0]->getArgumentTypes().empty()) + { + return false; + } + + // And cast it to the argument type of the first function in the chain + auto in_argument_type = functions[0]->getArgumentTypes()[0]; + if (canBeSafelyCasted(result_type, in_argument_type)) + { + result_column = castColumnAccurate({result_column, result_type, ""}, in_argument_type); + result_type = in_argument_type; + } + // If column cannot be casted accurate, casting with OrNull, and in case all + // values has been casted (no nulls), unpacking nested column from nullable. + // In case any further functions require Nullable input, they'll be able + // to cast it. + else + { + result_column = castColumnAccurateOrNull({result_column, result_type, ""}, in_argument_type); + const auto & result_column_nullable = assert_cast(*result_column); + const auto & null_map_data = result_column_nullable.getNullMapData(); + for (char8_t i : null_map_data) + { + if (i != 0) + return false; + } + result_column = result_column_nullable.getNestedColumnPtr(); + result_type = removeNullable(in_argument_type); + } + + for (const auto & func : functions) + { + if (func->getArgumentTypes().empty()) + return false; + + auto argument_type = func->getArgumentTypes()[0]; + if (!canBeSafelyCasted(result_type, argument_type)) + return false; + + result_column = castColumnAccurate({result_column, result_type, ""}, argument_type); + result_column = func->execute({{result_column, argument_type, ""}}, func->getResultType(), result_column->size()); + result_type = func->getResultType(); + + // Transforming nullable columns to the nested ones, in case no nulls found + if (result_column->isNullable()) + { + const auto & result_column_nullable = assert_cast(*result_column); + const auto & null_map_data = result_column_nullable.getNullMapData(); + for (char8_t i : null_map_data) + { + if (i != 0) + return false; + } + result_column = result_column_nullable.getNestedColumnPtr(); + result_type = removeNullable(func->getResultType()); + } + } + out_column = result_column; + out_data_type = result_type; + + return true; } bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( @@ -1118,13 +1006,13 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( if (out_value.isNull()) return false; - return transformConstantWithValidFunctions( + MonotonicFunctionsChain transform_functions; + auto can_transform_constant = extractMonotonicFunctionsChainFromKey( node.getTreeContext().getQueryContext(), expr_name, out_key_column_num, out_key_column_type, - out_value, - out_type, + transform_functions, [](const IFunctionBase & func, const IDataType & type) { if (!func.hasInformationAboutMonotonicity()) @@ -1138,6 +1026,27 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( } return true; }); + + if (!can_transform_constant) + return false; + + auto const_column = out_type->createColumnConst(1, out_value); + + ColumnPtr transformed_const_column; + DataTypePtr transformed_const_type; + bool constant_transformed = applyFunctionChainToColumn( + const_column, + out_type, + transform_functions, + transformed_const_column, + transformed_const_type); + + if (!constant_transformed) + return false; + + out_value = (*transformed_const_column)[0]; + out_type = transformed_const_type; + return true; } /// Looking for possible transformation of `column = constant` into `partition_expr = function(constant)` @@ -1173,61 +1082,34 @@ bool KeyCondition::canConstantBeWrappedByFunctions( if (out_value.isNull()) return false; - return transformConstantWithValidFunctions( + MonotonicFunctionsChain transform_functions; + auto can_transform_constant = extractMonotonicFunctionsChainFromKey( node.getTreeContext().getQueryContext(), expr_name, out_key_column_num, out_key_column_type, - out_value, + transform_functions, + [](const IFunctionBase & func, const IDataType &) { return func.isDeterministic(); }); + + if (!can_transform_constant) + return false; + + auto const_column = out_type->createColumnConst(1, out_value); + + ColumnPtr transformed_const_column; + DataTypePtr transformed_const_type; + bool constant_transformed = applyFunctionChainToColumn( + const_column, out_type, - [](const IFunctionBase & func, const IDataType &) - { - return func.isDeterministic(); - }); -} + transform_functions, + transformed_const_column, + transformed_const_type); -/// Sequentially applies functions to set column, returns `true` -/// if all function arguments are compatible with functions -/// signatures, and none of the functions produce `NULL` output. -/// -/// After functions chain execution, fills result set column and -/// its type. -bool applyFunctionChainToSetValues( - const ColumnPtr & set_column, - const DataTypePtr & set_data_type, - const std::vector & functions, - ColumnPtr & out_set_column, - DataTypePtr & out_set_type) -{ - auto result_column = set_column->convertToFullColumnIfLowCardinality(); - auto result_type = removeLowCardinality(set_data_type); - - for (const auto & func : functions) - { - if (func->getArgumentTypes().empty()) - return false; - - auto argument_type = func->getArgumentTypes()[0]; - if (!canBeSafelyCasted(result_type, argument_type)) - return false; - - result_column = castColumnAccurate({result_column, result_type, ""}, argument_type); - result_column = func->execute({{result_column, argument_type, ""}}, func->getResultType(), result_column->size()); - if (result_column->isNullable()) - { - const auto & result_column_nullable = assert_cast(*result_column); - const auto & null_map_data = result_column_nullable.getNullMapData(); - for (char8_t i : null_map_data) - { - if (i != 0) - return false; - } - } - result_type = func->getResultType(); - } - out_set_column = result_column; - out_set_type = result_type; + if (!constant_transformed) + return false; + out_value = (*transformed_const_column)[0]; + out_type = transformed_const_type; return true; } @@ -1260,6 +1142,7 @@ bool KeyCondition::tryPrepareSetIndex( out_key_column_num = std::max(out_key_column_num, index_mapping.key_index); set_transforming_chains.push_back(set_transforming_chain); } + // For partition index, checking if set can be transformed to prune any partitions else if (single_point && canSetValuesBeWrappedByFunctions(node, index_mapping.key_index, data_type, set_transforming_chain)) { indexes_mapping.push_back(index_mapping); @@ -1335,7 +1218,12 @@ bool KeyCondition::tryPrepareSetIndex( { ColumnPtr transformed_set_column; DataTypePtr transformed_set_type; - if (!applyFunctionChainToSetValues(set_column, set_element_type, set_transforming_chains[indexes_mapping_index], transformed_set_column, transformed_set_type)) + if (!applyFunctionChainToColumn( + set_column, + set_element_type, + set_transforming_chains[indexes_mapping_index], + transformed_set_column, + transformed_set_type)) return false; set_column = transformed_set_column; @@ -1639,36 +1527,54 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( return false; } -bool KeyCondition::canSetValuesBeWrappedByFunctions( - const RPNBuilderTreeNode & node, +/** When table's key has expression with these functions from a column, + * and when a column in a query is compared with a constant, such as: + * CREATE TABLE (x String) ORDER BY toDate(x) + * SELECT ... WHERE x LIKE 'Hello%' + * we want to apply the function to the constant for index analysis, + * but should modify it to pass on un-parsable values. + */ +static std::set date_time_parsing_functions = { + "toDate", + "toDate32", + "toDateTime", + "toDateTime64", + "parseDateTimeBestEffort", + "parseDateTimeBestEffortUS", + "parseDateTime32BestEffort", + "parseDateTime64BestEffort", + "parseDateTime", + "parseDateTimeInJodaSyntax", +}; + +/** The key functional expression constraint may be inferred from a plain column in the expression. + * For example, if the key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`, + * it can be assumed that if `toStartOfHour()` is monotonic on [now(), inf), the `toStartOfHour(Timestamp) >= toStartOfHour(now())` + * condition also holds, so the index may be used to select only parts satisfying this condition. + * + * To check the assumption, we'd need to assert that the inverse function to this transformation is also monotonic, however the + * inversion isn't exported (or even viable for not strictly monotonic functions such as `toStartOfHour()`). + * Instead, we can qualify only functions that do not transform the range (for example rounding), + * which while not strictly monotonic, are monotonic everywhere on the input range. + */ +bool KeyCondition::extractMonotonicFunctionsChainFromKey( + ContextPtr context, + const String & expr_name, size_t & out_key_column_num, - DataTypePtr & out_key_res_column_type, - MonotonicFunctionsChain & out_functions_chain) + DataTypePtr & out_key_column_type, + MonotonicFunctionsChain & out_functions_chain, + std::function always_monotonic) const { - // Checking if column name matches any of key subexpressions - String expr_name = node.getColumnName(); + const auto & sample_block = key_expr->getSampleBlock(); - if (array_joined_column_names.contains(expr_name)) - return false; - - if (!key_subexpr_names.contains(expr_name)) + for (const auto & node : key_expr->getNodes()) { - expr_name = node.getColumnNameWithModuloLegacy(); - - if (!key_subexpr_names.contains(expr_name)) - return false; - } - - // If match has been found, need to identify key column, and - // sequence of function nodes, which forms key column - for (const auto & key_expr_node : key_expr->getNodes()) - { - auto it = key_columns.find(key_expr_node.result_name); + auto it = key_columns.find(node.result_name); if (it != key_columns.end()) { - std::vector chain; + std::stack chain; - const auto * cur_node = &key_expr_node; + const auto * cur_node = &node; bool is_valid_chain = true; while (is_valid_chain) @@ -1676,10 +1582,11 @@ bool KeyCondition::canSetValuesBeWrappedByFunctions( if (cur_node->result_name == expr_name) break; + chain.push(cur_node); + if (cur_node->type == ActionsDAG::ActionType::FUNCTION && cur_node->children.size() <= 2) { - chain.push_back(cur_node); - is_valid_chain = cur_node->function_base->isDeterministic(); + is_valid_chain = always_monotonic(*cur_node->function_base, *cur_node->result_type); const ActionsDAG::Node * next_node = nullptr; for (const auto * arg : cur_node->children) @@ -1704,64 +1611,68 @@ bool KeyCondition::canSetValuesBeWrappedByFunctions( is_valid_chain = false; } - // If key column has been found, and it consists of deterministic function, - // in order to make them applicable to constants from query predicate, - // reversing the sequence of functions, and binding constant arguments if (is_valid_chain) { - // Removing low cardinality from `node` output type to avoid - // passing it through functions signatures - auto nonconst_input_type = removeLowCardinality(node.getDAGNode()->result_type); - - for (auto iter = chain.rbegin(); iter != chain.rend(); ++iter) + while (!chain.empty()) { - const auto * function_node = *iter; - auto function = function_node->function_base; - auto func_builder = FunctionFactory::instance().tryGet(function->getName(), node.getTreeContext().getQueryContext()); - if (!func_builder) - return false; + const auto * func = chain.top(); + chain.pop(); + + if (func->type != ActionsDAG::ActionType::FUNCTION) + continue; + + auto func_name = func->function_base->getName(); + auto func_base = func->function_base; + ColumnsWithTypeAndName arguments; ColumnWithTypeAndName const_arg; FunctionWithOptionalConstArg::Kind kind = FunctionWithOptionalConstArg::Kind::NO_CONST; - if (function_node->children.size() == 2) + + if (date_time_parsing_functions.contains(func_name)) { - const auto * left = function_node->children[0]; - const auto * right = function_node->children[1]; + const auto & arg_types = func_base->getArgumentTypes(); + if (!arg_types.empty() && isStringOrFixedString(arg_types[0])) + { + func_name = func_name + "OrNull"; + } + + } + + auto func_builder = FunctionFactory::instance().tryGet(func_name, context); + + if (func->children.size() == 1) + { + arguments.push_back({nullptr, removeLowCardinality(func->children[0]->result_type), ""}); + } + else if (func->children.size() == 2) + { + const auto * left = func->children[0]; + const auto * right = func->children[1]; if (left->column && isColumnConst(*left->column)) { const_arg = {left->result_type->createColumnConst(0, (*left->column)[0]), left->result_type, ""}; arguments.push_back(const_arg); - arguments.push_back({nullptr, nonconst_input_type, ""}); + arguments.push_back({nullptr, removeLowCardinality(right->result_type), ""}); kind = FunctionWithOptionalConstArg::Kind::LEFT_CONST; } - else if (right->column && isColumnConst(*right->column)) + else { const_arg = {right->result_type->createColumnConst(0, (*right->column)[0]), right->result_type, ""}; - arguments.push_back({nullptr, nonconst_input_type, ""}); + arguments.push_back({nullptr, removeLowCardinality(left->result_type), ""}); arguments.push_back(const_arg); kind = FunctionWithOptionalConstArg::Kind::RIGHT_CONST; } - - /// If constant arg of binary operator is NULL, there will be no monotonicity. - if (const_arg.column->isNullAt(0)) - return false; } - else - arguments.push_back({nullptr, nonconst_input_type, ""}); - auto func = func_builder->build(arguments); + auto out_func = func_builder->build(arguments); if (kind == FunctionWithOptionalConstArg::Kind::NO_CONST) - out_functions_chain.push_back(func); + out_functions_chain.push_back(out_func); else - out_functions_chain.push_back(std::make_shared(func, const_arg, kind)); - - nonconst_input_type = func->getResultType(); + out_functions_chain.push_back(std::make_shared(out_func, const_arg, kind)); } - const auto & sample_block = key_expr->getSampleBlock(); out_key_column_num = it->second; - out_key_res_column_type = sample_block.getByName(it->first).type; - + out_key_column_type = sample_block.getByName(it->first).type; return true; } } @@ -1770,6 +1681,38 @@ bool KeyCondition::canSetValuesBeWrappedByFunctions( return false; } +bool KeyCondition::canSetValuesBeWrappedByFunctions( + const RPNBuilderTreeNode & node, + size_t & out_key_column_num, + DataTypePtr & out_key_res_column_type, + MonotonicFunctionsChain & out_functions_chain) +{ + // Checking if column name matches any of key subexpressions + String expr_name = node.getColumnName(); + + if (array_joined_column_names.contains(expr_name)) + return false; + + if (!key_subexpr_names.contains(expr_name)) + { + expr_name = node.getColumnNameWithModuloLegacy(); + + if (!key_subexpr_names.contains(expr_name)) + return false; + } + + return extractMonotonicFunctionsChainFromKey( + node.getTreeContext().getQueryContext(), + expr_name, + out_key_column_num, + out_key_res_column_type, + out_functions_chain, + [](const IFunctionBase & func, const IDataType &) + { + return func.isDeterministic(); + }); +} + static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const String & node_column_name) { try diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 11dd1c85c9d..9e2218d7a29 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -14,6 +14,7 @@ #include #include +#include "DataTypes/Serializations/ISerialization.h" namespace DB @@ -253,13 +254,12 @@ private: DataTypePtr & out_key_column_type, std::vector & out_functions_chain); - bool transformConstantWithValidFunctions( + bool extractMonotonicFunctionsChainFromKey( ContextPtr context, const String & expr_name, size_t & out_key_column_num, DataTypePtr & out_key_column_type, - Field & out_value, - DataTypePtr & out_type, + MonotonicFunctionsChain & out_functions_chain, std::function always_monotonic) const; bool canConstantBeWrappedByMonotonicFunctions(