diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 06fa8e83505..268c45c305f 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -592,24 +592,15 @@ void KeyCondition::traverseAST(const ASTPtr & node, ContextPtr context, Block & rpn.emplace_back(std::move(element)); } -bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( - const ASTPtr & node, - size_t & out_key_column_num, - DataTypePtr & out_key_column_type, - Field & out_value, - DataTypePtr & out_type) +bool KeyCondition::canConstantBeWrapped(const ASTPtr & node, const String & expr_name, String & result_expr_name) { const auto & sample_block = key_expr->getSampleBlock(); - // Constant expr should use alias names if any - String passed_expr_name = node->getColumnName(); - String expr_name; - /// sample_block from key_expr cannot contain modulo and moduloLegacy at the same time. /// For partition key it is always moduloLegacy. - if (sample_block.has(passed_expr_name)) + if (sample_block.has(expr_name)) { - expr_name = passed_expr_name; + result_expr_name = expr_name; } else { @@ -620,9 +611,27 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( if (!sample_block.has(adjusted_expr_name)) return false; - expr_name = adjusted_expr_name; + result_expr_name = adjusted_expr_name; } + return true; +} + +bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( + const ASTPtr & node, + size_t & out_key_column_num, + DataTypePtr & out_key_column_type, + Field & out_value, + DataTypePtr & out_type) +{ + // Constant expr should use alias names if any + String passed_expr_name = node->getColumnName(); + String expr_name; + if (!canConstantBeWrapped(node, passed_expr_name, expr_name)) + return false; + + const auto & sample_block = key_expr->getSampleBlock(); + /// TODO Nullable index is not yet landed. if (out_value.isNull()) return false; @@ -685,29 +694,13 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( bool KeyCondition::canConstantBeWrappedByFunctions( const ASTPtr & ast, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type) { - const auto & sample_block = key_expr->getSampleBlock(); - // Constant expr should use alias names if any String passed_expr_name = ast->getColumnName(); String expr_name; + if (!canConstantBeWrapped(ast, passed_expr_name, expr_name)) + return false; - /// sample_block from key_expr cannot contain modulo and moduloLegacy at the same time. - /// For partition key it is always moduloLegacy. - if (sample_block.has(passed_expr_name)) - { - expr_name = passed_expr_name; - } - else - { - auto adjusted_ast = ast->clone(); - KeyDescription::moduloToModuloLegacyRecursive(adjusted_ast); - String adjusted_expr_name = adjusted_ast->getColumnName(); - - if (!sample_block.has(adjusted_expr_name)) - return false; - - expr_name = adjusted_expr_name; - } + const auto & sample_block = key_expr->getSampleBlock(); /// TODO Nullable index is not yet landed. if (out_value.isNull()) diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index bd51769ad1f..fc28d4a93c9 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -419,6 +419,12 @@ private: bool canConstantBeWrappedByFunctions( const ASTPtr & ast, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type); + /// Check if ASTPtr node, passed to canConstantBeWrappedBy*, can be used by them for further checks. + /// Always call this method at start of other methods, which require key comparison, because it also checks if adjusted + /// key expression can also be used (with substitution from modulo to moduloLegacy). This is needed because partition key + /// is always modified, when passed into keyCondition, - with recursive substitution from modulo to moduloLegacy. + bool canConstantBeWrapped(const ASTPtr & node, const String & expr_name, String & result_expr_name); + /// 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. diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 3f128ef03a3..3bf33d45802 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -157,11 +157,11 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts( Block block_copy = block; /// After expression execution partition key columns will be added to block_copy with names regarding partition function. - auto partition_key_sample_block = MergeTreePartition::executePartitionByExpression(metadata_snapshot, block_copy, context); + auto partition_key_names_and_types = MergeTreePartition::executePartitionByExpression(metadata_snapshot, block_copy, context); ColumnRawPtrs partition_columns; - partition_columns.reserve(partition_key_sample_block.columns()); - for (const ColumnWithTypeAndName & element : partition_key_sample_block) + partition_columns.reserve(partition_key_names_and_types.size()); + for (const auto & element : partition_key_names_and_types) partition_columns.emplace_back(block_copy.getByName(element.name).column.get()); PODArray partition_num_to_first_row; diff --git a/src/Storages/MergeTree/MergeTreePartition.cpp b/src/Storages/MergeTree/MergeTreePartition.cpp index 8ad48ab0516..a19bd263dbf 100644 --- a/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/src/Storages/MergeTree/MergeTreePartition.cpp @@ -165,31 +165,31 @@ void MergeTreePartition::create(const StorageMetadataPtr & metadata_snapshot, Bl if (!metadata_snapshot->hasPartitionKey()) return; - auto partition_key_sample_block = executePartitionByExpression(metadata_snapshot, block, context); - size_t partition_columns_num = partition_key_sample_block.columns(); - value.resize(partition_columns_num); - const String modulo_legacy_function_name = "moduloLegacy"; + auto partition_key_names_and_types = executePartitionByExpression(metadata_snapshot, block, context); + value.resize(partition_key_names_and_types.size()); - for (size_t i = 0; i < partition_columns_num; ++i) + /// Executing partition_by expression adds new columns to passed block according to partition functions. + /// The block is passed by reference and is used afterwards. `moduloLegacy` needs to be substituted back + /// with just `modulo`, because it was a temporary substitution. + static constexpr auto modulo_legacy_function_name = "moduloLegacy"; + + size_t i = 0; + for (const auto & element : partition_key_names_and_types) { - const auto & column_name = partition_key_sample_block.getByPosition(i).name; - auto & partition_column = block.getByName(column_name); + auto & partition_column = block.getByName(element.name); - /// Executing partition_by expression adds new columns to passed block according to partition functions. - /// The block is passed by reference and is used afterwards. `moduloLegacy` needs to be substituted back - /// with just `modulo`, because it was a temporary substitution. - if (column_name.starts_with(modulo_legacy_function_name)) - partition_column.name = "modulo" + partition_column.name.substr(modulo_legacy_function_name.size()); + if (element.name.starts_with(modulo_legacy_function_name)) + partition_column.name = "modulo" + partition_column.name.substr(std::strlen(modulo_legacy_function_name)); - partition_column.column->get(row, value[i]); + partition_column.column->get(row, value[i++]); } } -Block MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) +NamesAndTypesList MergeTreePartition::executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context) { auto adjusted_partition_key = adjustPartitionKey(metadata_snapshot, context); adjusted_partition_key.expression->execute(block); - return adjusted_partition_key.sample_block; + return adjusted_partition_key.sample_block.getNamesAndTypesList(); } KeyDescription MergeTreePartition::adjustPartitionKey(const StorageMetadataPtr & metadata_snapshot, ContextPtr context) diff --git a/src/Storages/MergeTree/MergeTreePartition.h b/src/Storages/MergeTree/MergeTreePartition.h index 541881633a0..67e7a246c30 100644 --- a/src/Storages/MergeTree/MergeTreePartition.h +++ b/src/Storages/MergeTree/MergeTreePartition.h @@ -44,7 +44,7 @@ public: void create(const StorageMetadataPtr & metadata_snapshot, Block block, size_t row, ContextPtr context); /// Adjust partition key and execute its expression on block. Return sample block according to used expression. - static Block executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context); + static NamesAndTypesList executePartitionByExpression(const StorageMetadataPtr & metadata_snapshot, Block & block, ContextPtr context); /// Make a modified partition key with substitution from modulo to moduloLegacy. Used in paritionPruner. static KeyDescription adjustPartitionKey(const StorageMetadataPtr & metadata_snapshot, ContextPtr context); diff --git a/tests/queries/0_stateless/01870_modulo_partition_key.reference b/tests/queries/0_stateless/01870_modulo_partition_key.reference index 29142711066..51c629e0a50 100644 --- a/tests/queries/0_stateless/01870_modulo_partition_key.reference +++ b/tests/queries/0_stateless/01870_modulo_partition_key.reference @@ -105,6 +105,8 @@ After detach: (58,8,29) (59,9,30) (60,0,30) +Indexes: +100 comparison: 0 -205 -5 -5 1 -204 -4 -4 diff --git a/tests/queries/0_stateless/01870_modulo_partition_key.sql b/tests/queries/0_stateless/01870_modulo_partition_key.sql index b3fdfde3a88..344b32659a8 100644 --- a/tests/queries/0_stateless/01870_modulo_partition_key.sql +++ b/tests/queries/0_stateless/01870_modulo_partition_key.sql @@ -1,11 +1,12 @@ SELECT 'simple partition key:'; DROP TABLE IF EXISTS table1 SYNC; CREATE TABLE table1 (id Int64, v UInt64) -ENGINE = ReplicatedReplacingMergeTree('/clickhouse/test/tables/table', '1', v) +ENGINE = ReplicatedReplacingMergeTree('/clickhouse/test/tables/table12', '1', v) PARTITION BY id % 200 ORDER BY id; INSERT INTO table1 SELECT number-205, number FROM numbers(10); INSERT INTO table1 SELECT number-205, number FROM numbers(400, 10); -SELECT toInt64(partition) as p FROM system.parts WHERE table='table1' ORDER BY p; +SELECT toInt64(partition) as p FROM system.parts WHERE table='table1' and database=currentDatabase() ORDER BY p; + select 'where id % 200 = +-2:'; select id from table1 where id % 200 = 2 OR id % 200 = -2 order by id; select 'where id % 200 > 0:'; @@ -20,7 +21,7 @@ ENGINE = MergeTree() PARTITION BY (toInt32(id / 2) % 3, id % 200) ORDER BY id; INSERT INTO table2 SELECT number-205, number FROM numbers(10); INSERT INTO table2 SELECT number-205, number FROM numbers(400, 10); -SELECT partition as p FROM system.parts WHERE table='table2' ORDER BY p; +SELECT partition as p FROM system.parts WHERE table='table2' and database=currentDatabase() ORDER BY p; SELECT 'recursive modulo partition key:'; DROP TABLE IF EXISTS table3; @@ -29,11 +30,20 @@ ENGINE = MergeTree() PARTITION BY (id % 200, (id % 200) % 10, toInt32(round((id % 200) / 2, 0))) ORDER BY id; INSERT INTO table3 SELECT number-205, number FROM numbers(10); INSERT INTO table3 SELECT number-205, number FROM numbers(400, 10); -SELECT partition as p FROM system.parts WHERE table='table3' ORDER BY p; +SELECT partition as p FROM system.parts WHERE table='table3' and database=currentDatabase() ORDER BY p; + DETACH TABLE table3; ATTACH TABLE table3; SELECT 'After detach:'; -SELECT partition as p FROM system.parts WHERE table='table3' ORDER BY p; +SELECT partition as p FROM system.parts WHERE table='table3' and database=currentDatabase() ORDER BY p; + +SELECT 'Indexes:'; +DROP TABLE IF EXISTS table4; +CREATE TABLE table4 (id Int64, v UInt64, s String, +INDEX a (id * 2, s) TYPE minmax GRANULARITY 3 +) ENGINE = MergeTree() PARTITION BY id % 10 ORDER BY v; +INSERT INTO table4 SELECT number, number, toString(number) FROM numbers(1000); +SELECT count() FROM table4 WHERE id % 10 = 7; SELECT 'comparison:'; SELECT v, v-205 as vv, modulo(vv, 200), moduloLegacy(vv, 200) FROM table1 ORDER BY v;