From 8ea4c2e26fbf51e8aa59f6ea3bf4e9b366182d67 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 24 Sep 2020 14:42:41 +0300 Subject: [PATCH 01/12] Fix TTL in cases, when its expression is a function and is the same as ORDER BY key --- src/DataStreams/TTLBlockInputStream.cpp | 56 ++++++------- .../MergeTree/MergeTreeDataWriter.cpp | 42 ++++------ src/Storages/TTLDescription.cpp | 49 ++---------- .../01506_ttl_same_with_order_by.reference | 4 + .../01506_ttl_same_with_order_by.sql | 78 +++++++++++++++++++ 5 files changed, 128 insertions(+), 101 deletions(-) create mode 100644 tests/queries/0_stateless/01506_ttl_same_with_order_by.reference create mode 100644 tests/queries/0_stateless/01506_ttl_same_with_order_by.sql diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 85d9c7fead2..6dba8968f79 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -86,6 +86,7 @@ TTLBlockInputStream::TTLBlockInputStream( if (descr.arguments.empty()) for (const auto & name : descr.argument_names) descr.arguments.push_back(header.getPositionByName(name)); + agg_aggregate_columns.resize(storage_rows_ttl.aggregate_descriptions.size()); const Settings & settings = storage.global_context.getSettingsRef(); @@ -153,19 +154,26 @@ void TTLBlockInputStream::readSuffixImpl() LOG_INFO(log, "Removed {} rows with expired TTL from part {}", rows_removed, data_part->name); } +static ColumnPtr extractRequieredColumn(const ExpressionActions & expression, const Block & block, const String & result_column) +{ + if (block.has(result_column)) + return block.getByName(result_column).column; + + Block block_copy; + for (const auto & column_name : expression.getRequiredColumns()) + block_copy.insert(block.getByName(column_name)); + + expression.execute(block_copy); + return block_copy.getByName(result_column).column; +} + void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) { auto rows_ttl = metadata_snapshot->getRowsTTL(); + auto ttl_column = extractRequieredColumn(*rows_ttl.expression, block, rows_ttl.result_column); - rows_ttl.expression->execute(block); - if (rows_ttl.where_expression) - rows_ttl.where_expression->execute(block); - - const IColumn * ttl_column = - block.getByName(rows_ttl.result_column).column.get(); - - const IColumn * where_result_column = rows_ttl.where_expression ? - block.getByName(rows_ttl.where_result_column).column.get() : nullptr; + auto where_result_column = rows_ttl.where_expression ? + extractRequieredColumn(*rows_ttl.where_expression, block, rows_ttl.where_result_column) : nullptr; const auto & column_names = header.getNames(); @@ -181,7 +189,7 @@ void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) for (size_t i = 0; i < block.rows(); ++i) { - UInt32 cur_ttl = getTimestampByIndex(ttl_column, i); + UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); bool where_filter_passed = !where_result_column || where_result_column->getBool(i); if (!isTTLExpired(cur_ttl) || !where_filter_passed) { @@ -206,7 +214,7 @@ void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) auto storage_rows_ttl = metadata_snapshot->getRowsTTL(); for (size_t i = 0; i < block.rows(); ++i) { - UInt32 cur_ttl = getTimestampByIndex(ttl_column, i); + UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); bool where_filter_passed = !where_result_column || where_result_column->getBool(i); bool ttl_expired = isTTLExpired(cur_ttl) && where_filter_passed; @@ -221,6 +229,7 @@ void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) same_as_current = false; } } + if (!same_as_current) { if (rows_with_current_key) @@ -311,7 +320,6 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) defaults_expression->execute(block_with_defaults); } - std::vector columns_to_remove; for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs()) { /// If we read not all table columns. E.g. while mutation. @@ -329,11 +337,7 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) if (isTTLExpired(old_ttl_info.max)) continue; - if (!block.has(ttl_entry.result_column)) - { - columns_to_remove.push_back(ttl_entry.result_column); - ttl_entry.expression->execute(block); - } + auto ttl_column = extractRequieredColumn(*ttl_entry.expression, block, ttl_entry.result_column); ColumnPtr default_column = nullptr; if (block_with_defaults.has(name)) @@ -344,11 +348,9 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) MutableColumnPtr result_column = values_column->cloneEmpty(); result_column->reserve(block.rows()); - const IColumn * ttl_column = block.getByName(ttl_entry.result_column).column.get(); - for (size_t i = 0; i < block.rows(); ++i) { - UInt32 cur_ttl = getTimestampByIndex(ttl_column, i); + UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); if (isTTLExpired(cur_ttl)) { if (default_column) @@ -365,34 +367,24 @@ void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) } column_with_type.column = std::move(result_column); } - - for (const String & column : columns_to_remove) - block.erase(column); } void TTLBlockInputStream::updateTTLWithDescriptions(Block & block, const TTLDescriptions & descriptions, TTLInfoMap & ttl_info_map) { - std::vector columns_to_remove; for (const auto & ttl_entry : descriptions) { auto & new_ttl_info = ttl_info_map[ttl_entry.result_column]; if (!block.has(ttl_entry.result_column)) - { - columns_to_remove.push_back(ttl_entry.result_column); ttl_entry.expression->execute(block); - } - const IColumn * ttl_column = block.getByName(ttl_entry.result_column).column.get(); + auto ttl_column = extractRequieredColumn(*ttl_entry.expression, block, ttl_entry.result_column); for (size_t i = 0; i < block.rows(); ++i) { - UInt32 cur_ttl = getTimestampByIndex(ttl_column, i); + UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); new_ttl_info.update(cur_ttl); } } - - for (const String & column : columns_to_remove) - block.erase(column); } void TTLBlockInputStream::updateMovesTTL(Block & block) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 739aff31a06..d5a2bfe280e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -84,19 +84,14 @@ void updateTTL( const TTLDescription & ttl_entry, IMergeTreeDataPart::TTLInfos & ttl_infos, DB::MergeTreeDataPartTTLInfo & ttl_info, - Block & block, + const Block & block, bool update_part_min_max_ttls) { - bool remove_column = false; - if (!block.has(ttl_entry.result_column)) - { - ttl_entry.expression->execute(block); - remove_column = true; - } + Block block_copy = block; + if (!block_copy.has(ttl_entry.result_column)) + ttl_entry.expression->execute(block_copy); - const auto & current = block.getByName(ttl_entry.result_column); - - const IColumn * column = current.column.get(); + const IColumn * column = block_copy.getByName(ttl_entry.result_column).column.get(); if (const ColumnUInt16 * column_date = typeid_cast(column)) { const auto & date_lut = DateLUT::instance(); @@ -127,9 +122,6 @@ void updateTTL( if (update_part_min_max_ttls) ttl_infos.updatePartMinMaxTTL(ttl_info.min, ttl_info.max); - - if (remove_column) - block.erase(ttl_entry.result_column); } } @@ -271,6 +263,18 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa sync_guard.emplace(disk, full_path); } + if (metadata_snapshot->hasRowsTTL()) + updateTTL(metadata_snapshot->getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true); + + for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs()) + updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true); + + const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); + for (const auto & ttl_entry : recompression_ttl_entries) + updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.recompression_ttl[ttl_entry.result_column], block, false); + + new_data_part->ttl_infos.update(move_ttl_infos); + /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) data.getSortingKeyAndSkipIndicesExpression(metadata_snapshot)->execute(block); @@ -299,18 +303,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa ProfileEvents::increment(ProfileEvents::MergeTreeDataWriterBlocksAlreadySorted); } - if (metadata_snapshot->hasRowsTTL()) - updateTTL(metadata_snapshot->getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true); - - for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs()) - updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true); - - const auto & recompression_ttl_entries = metadata_snapshot->getRecompressionTTLs(); - for (const auto & ttl_entry : recompression_ttl_entries) - updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.recompression_ttl[ttl_entry.result_column], block, false); - - new_data_part->ttl_infos.update(move_ttl_infos); - /// This effectively chooses minimal compression method: /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.global_context.chooseCompressionCodec(0, 0); diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 7f55badf819..7499f1de292 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -10,6 +10,8 @@ #include #include +#include + #include #include @@ -196,59 +198,20 @@ TTLDescription TTLDescription::getTTLFromAST( ErrorCodes::BAD_TTL_EXPRESSION); } - for (const auto & [name, value] : ttl_element->group_by_aggregations) - { - if (primary_key_columns_set.count(name)) - throw Exception( - "Can not set custom aggregation for column in primary key in TTL Expression", - ErrorCodes::BAD_TTL_EXPRESSION); - + for (const auto & [name, _] : ttl_element->group_by_aggregations) aggregation_columns_set.insert(name); - } if (aggregation_columns_set.size() != ttl_element->group_by_aggregations.size()) throw Exception( "Multiple aggregations set for one column in TTL Expression", ErrorCodes::BAD_TTL_EXPRESSION); - result.group_by_keys = Names(pk_columns.begin(), pk_columns.begin() + ttl_element->group_by_key.size()); - auto aggregations = ttl_element->group_by_aggregations; - for (size_t i = 0; i < pk_columns.size(); ++i) + for (const auto & column : columns.getOrdinary()) { - ASTPtr value = primary_key.expression_list_ast->children[i]->clone(); - - if (i >= ttl_element->group_by_key.size()) - { - ASTPtr value_max = makeASTFunction("max", value->clone()); - aggregations.emplace_back(value->getColumnName(), std::move(value_max)); - } - - if (value->as()) - { - auto syntax_result = TreeRewriter(context).analyze(value, columns.getAllPhysical(), {}, {}, true); - auto expr_actions = ExpressionAnalyzer(value, syntax_result, context).getActions(false); - for (const auto & column : expr_actions->getRequiredColumns()) - { - if (i < ttl_element->group_by_key.size()) - { - ASTPtr expr = makeASTFunction("any", std::make_shared(column)); - aggregations.emplace_back(column, std::move(expr)); - } - else - { - ASTPtr expr = makeASTFunction("argMax", std::make_shared(column), value->clone()); - aggregations.emplace_back(column, std::move(expr)); - } - } - } - } - - for (const auto & column : columns.getAllPhysical()) - { - if (!primary_key_columns_set.count(column.name) && !aggregation_columns_set.count(column.name)) + if (!aggregation_columns_set.count(column.name)) { ASTPtr expr = makeASTFunction("any", std::make_shared(column.name)); aggregations.emplace_back(column.name, std::move(expr)); @@ -280,8 +243,6 @@ TTLDescription TTLDescription::getTTLFromAST( } checkTTLExpression(result.expression, result.result_column); - - return result; } diff --git a/tests/queries/0_stateless/01506_ttl_same_with_order_by.reference b/tests/queries/0_stateless/01506_ttl_same_with_order_by.reference new file mode 100644 index 00000000000..f8f36434a82 --- /dev/null +++ b/tests/queries/0_stateless/01506_ttl_same_with_order_by.reference @@ -0,0 +1,4 @@ +2020-01-01 00:00:00 3 +2020-01-01 00:00:00 2020-01-01 00:00:00 111 +1 +0 diff --git a/tests/queries/0_stateless/01506_ttl_same_with_order_by.sql b/tests/queries/0_stateless/01506_ttl_same_with_order_by.sql new file mode 100644 index 00000000000..7a0fb86330b --- /dev/null +++ b/tests/queries/0_stateless/01506_ttl_same_with_order_by.sql @@ -0,0 +1,78 @@ +DROP TABLE IF EXISTS derived_metrics_local; + +CREATE TABLE derived_metrics_local +( + timestamp DateTime, + bytes UInt64 +) +ENGINE=SummingMergeTree() +PARTITION BY toYYYYMMDD(timestamp) +ORDER BY (toStartOfHour(timestamp), timestamp) +TTL toStartOfHour(timestamp) + INTERVAL 1 HOUR GROUP BY toStartOfHour(timestamp) +SET bytes=max(bytes); + +INSERT INTO derived_metrics_local values('2020-01-01 00:00:00', 1); +INSERT INTO derived_metrics_local values('2020-01-01 00:01:00', 3); +INSERT INTO derived_metrics_local values('2020-01-01 00:02:00', 2); + +OPTIMIZE TABLE derived_metrics_local FINAL; +SELECT * FROM derived_metrics_local; + +DROP TABLE derived_metrics_local; + +CREATE TABLE derived_metrics_local +( + timestamp DateTime, + timestamp_h DateTime materialized toStartOfHour(timestamp), + bytes UInt64 +) +ENGINE=SummingMergeTree() +PARTITION BY toYYYYMMDD(timestamp) +ORDER BY (timestamp_h, timestamp) +TTL toStartOfHour(timestamp) + INTERVAL 1 HOUR GROUP BY timestamp_h +SET bytes=max(bytes), timestamp = toStartOfHour(any(timestamp)); + +INSERT INTO derived_metrics_local values('2020-01-01 00:01:00', 111); +INSERT INTO derived_metrics_local values('2020-01-01 00:19:22', 22); +INSERT INTO derived_metrics_local values('2020-01-01 00:59:02', 1); + +OPTIMIZE TABLE derived_metrics_local FINAL; +SELECT timestamp, timestamp_h, bytes FROM derived_metrics_local; + +DROP TABLE IF EXISTS derived_metrics_local; + +CREATE TABLE derived_metrics_local +( + timestamp DateTime, + bytes UInt64 TTL toStartOfHour(timestamp) + INTERVAL 1 HOUR +) +ENGINE=MergeTree() +ORDER BY (toStartOfHour(timestamp), timestamp) +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO derived_metrics_local values('2020-01-01 00:01:00', 111) ('2020-01-01 00:19:22', 22) ('2100-01-01 00:19:22', 1); + +OPTIMIZE TABLE derived_metrics_local FINAL; +SELECT sum(bytes) FROM derived_metrics_local; + +DROP TABLE IF EXISTS derived_metrics_local; + +CREATE TABLE derived_metrics_local +( + timestamp DateTime, + bytes UInt64 +) +ENGINE=MergeTree() +PARTITION BY toYYYYMMDD(timestamp) +ORDER BY (toStartOfHour(timestamp), timestamp) +TTL toStartOfHour(timestamp) + INTERVAL 1 HOUR +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO derived_metrics_local values('2020-01-01 00:01:00', 111); +INSERT INTO derived_metrics_local values('2020-01-01 00:19:22', 22); +INSERT INTO derived_metrics_local values('2020-01-01 00:59:02', 1); + +OPTIMIZE TABLE derived_metrics_local FINAL; +SELECT count() FROM derived_metrics_local; + +DROP TABLE IF EXISTS derived_metrics_local; From 0d79474acc3cfb6f2c8dfbed26aa5d5f0346fc4f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 30 Sep 2020 19:10:15 +0300 Subject: [PATCH 02/12] Fix TTL with GROUP BY and fix test according to new logic --- src/DataStreams/TTLBlockInputStream.cpp | 184 ++++++++++-------- src/DataStreams/TTLBlockInputStream.h | 7 +- src/Storages/TTLDescription.cpp | 17 +- .../01280_ttl_where_group_by.reference | 16 +- .../0_stateless/01280_ttl_where_group_by.sh | 8 +- .../01280_ttl_where_group_by_negative.sql | 3 - 6 files changed, 134 insertions(+), 101 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 6dba8968f79..5c49b9f11c2 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -88,7 +88,6 @@ TTLBlockInputStream::TTLBlockInputStream( descr.arguments.push_back(header.getPositionByName(name)); agg_aggregate_columns.resize(storage_rows_ttl.aggregate_descriptions.size()); - const Settings & settings = storage.global_context.getSettingsRef(); Aggregator::Params params(header, keys, aggregates, @@ -108,14 +107,15 @@ Block TTLBlockInputStream::readImpl() { /// Skip all data if table ttl is expired for part auto storage_rows_ttl = metadata_snapshot->getRowsTTL(); - if (metadata_snapshot->hasRowsTTL() && !storage_rows_ttl.where_expression && storage_rows_ttl.mode != TTLMode::GROUP_BY + if (metadata_snapshot->hasRowsTTL() + && !storage_rows_ttl.where_expression + && storage_rows_ttl.mode != TTLMode::GROUP_BY && isTTLExpired(old_ttl_infos.table_ttl.max)) { rows_removed = data_part->rows_count; return {}; } - Block block = children.at(0)->read(); if (!block) { @@ -130,10 +130,9 @@ Block TTLBlockInputStream::readImpl() } if (metadata_snapshot->hasRowsTTL() && (force || isTTLExpired(old_ttl_infos.table_ttl.min))) - removeRowsWithExpiredTableTTL(block); + executeRowsTTL(block); removeValuesWithExpiredColumnTTL(block); - updateMovesTTL(block); updateRecompressionTTL(block); @@ -167,107 +166,117 @@ static ColumnPtr extractRequieredColumn(const ExpressionActions & expression, co return block_copy.getByName(result_column).column; } -void TTLBlockInputStream::removeRowsWithExpiredTableTTL(Block & block) +void TTLBlockInputStream::executeRowsTTL(Block & block) { auto rows_ttl = metadata_snapshot->getRowsTTL(); auto ttl_column = extractRequieredColumn(*rows_ttl.expression, block, rows_ttl.result_column); auto where_result_column = rows_ttl.where_expression ? - extractRequieredColumn(*rows_ttl.where_expression, block, rows_ttl.where_result_column) : nullptr; + extractRequieredColumn(*rows_ttl.where_expression, block, rows_ttl.where_result_column): nullptr; + if (aggregator) + aggregateRowsWithExpiredTTL(block, ttl_column, where_result_column); + else + removeRowsWithExpiredTTL(block, ttl_column, where_result_column); +} + +void TTLBlockInputStream::removeRowsWithExpiredTTL(Block & block, ColumnPtr ttl_column, ColumnPtr where_column) +{ + MutableColumns result_columns; const auto & column_names = header.getNames(); - if (!aggregator) + result_columns.reserve(column_names.size()); + for (auto it = column_names.begin(); it != column_names.end(); ++it) { - MutableColumns result_columns; - result_columns.reserve(column_names.size()); - for (auto it = column_names.begin(); it != column_names.end(); ++it) - { - const IColumn * values_column = block.getByName(*it).column.get(); - MutableColumnPtr result_column = values_column->cloneEmpty(); - result_column->reserve(block.rows()); + const IColumn * values_column = block.getByName(*it).column.get(); + MutableColumnPtr result_column = values_column->cloneEmpty(); + result_column->reserve(block.rows()); - for (size_t i = 0; i < block.rows(); ++i) - { - UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); - bool where_filter_passed = !where_result_column || where_result_column->getBool(i); - if (!isTTLExpired(cur_ttl) || !where_filter_passed) - { - new_ttl_infos.table_ttl.update(cur_ttl); - result_column->insertFrom(*values_column, i); - } - else if (it == column_names.begin()) - ++rows_removed; - } - result_columns.emplace_back(std::move(result_column)); - } - block = header.cloneWithColumns(std::move(result_columns)); - } - else - { - MutableColumns result_columns = header.cloneEmptyColumns(); - MutableColumns aggregate_columns = header.cloneEmptyColumns(); - - size_t rows_aggregated = 0; - size_t current_key_start = 0; - size_t rows_with_current_key = 0; - auto storage_rows_ttl = metadata_snapshot->getRowsTTL(); for (size_t i = 0; i < block.rows(); ++i) { UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); - bool where_filter_passed = !where_result_column || where_result_column->getBool(i); - bool ttl_expired = isTTLExpired(cur_ttl) && where_filter_passed; - - bool same_as_current = true; - for (size_t j = 0; j < storage_rows_ttl.group_by_keys.size(); ++j) - { - const String & key_column = storage_rows_ttl.group_by_keys[j]; - const IColumn * values_column = block.getByName(key_column).column.get(); - if (!same_as_current || (*values_column)[i] != current_key_value[j]) - { - values_column->get(i, current_key_value[j]); - same_as_current = false; - } - } - - if (!same_as_current) - { - if (rows_with_current_key) - calculateAggregates(aggregate_columns, current_key_start, rows_with_current_key); - finalizeAggregates(result_columns); - - current_key_start = rows_aggregated; - rows_with_current_key = 0; - } - - if (ttl_expired) - { - ++rows_with_current_key; - ++rows_aggregated; - for (const auto & name : column_names) - { - const IColumn * values_column = block.getByName(name).column.get(); - auto & column = aggregate_columns[header.getPositionByName(name)]; - column->insertFrom(*values_column, i); - } - } - else + bool where_filter_passed = !where_column || where_column->getBool(i); + if (!isTTLExpired(cur_ttl) || !where_filter_passed) { new_ttl_infos.table_ttl.update(cur_ttl); - for (const auto & name : column_names) - { - const IColumn * values_column = block.getByName(name).column.get(); - auto & column = result_columns[header.getPositionByName(name)]; - column->insertFrom(*values_column, i); - } + result_column->insertFrom(*values_column, i); + } + else if (it == column_names.begin()) + ++rows_removed; + } + + result_columns.emplace_back(std::move(result_column)); + } + + block = header.cloneWithColumns(std::move(result_columns)); +} + +void TTLBlockInputStream::aggregateRowsWithExpiredTTL(Block & block, ColumnPtr ttl_column, ColumnPtr where_column) +{ + const auto & column_names = header.getNames(); + MutableColumns result_columns = header.cloneEmptyColumns(); + MutableColumns aggregate_columns = header.cloneEmptyColumns(); + + size_t rows_aggregated = 0; + size_t current_key_start = 0; + size_t rows_with_current_key = 0; + auto storage_rows_ttl = metadata_snapshot->getRowsTTL(); + + for (size_t i = 0; i < block.rows(); ++i) + { + UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); + bool where_filter_passed = !where_column || where_column->getBool(i); + bool ttl_expired = isTTLExpired(cur_ttl) && where_filter_passed; + + bool same_as_current = true; + for (size_t j = 0; j < storage_rows_ttl.group_by_keys.size(); ++j) + { + const String & key_column = storage_rows_ttl.group_by_keys[j]; + const IColumn * values_column = block.getByName(key_column).column.get(); + if (!same_as_current || (*values_column)[i] != current_key_value[j]) + { + values_column->get(i, current_key_value[j]); + same_as_current = false; } } - if (rows_with_current_key) - calculateAggregates(aggregate_columns, current_key_start, rows_with_current_key); + if (!same_as_current) + { + if (rows_with_current_key) + calculateAggregates(aggregate_columns, current_key_start, rows_with_current_key); + finalizeAggregates(result_columns); - block = header.cloneWithColumns(std::move(result_columns)); + current_key_start = rows_aggregated; + rows_with_current_key = 0; + } + + if (ttl_expired) + { + ++rows_with_current_key; + ++rows_aggregated; + for (const auto & name : column_names) + { + const IColumn * values_column = block.getByName(name).column.get(); + auto & column = aggregate_columns[header.getPositionByName(name)]; + column->insertFrom(*values_column, i); + } + } + else + { + new_ttl_infos.table_ttl.update(cur_ttl); + for (const auto & name : column_names) + { + const IColumn * values_column = block.getByName(name).column.get(); + auto & column = result_columns[header.getPositionByName(name)]; + column->insertFrom(*values_column, i); + } + } } + + if (rows_with_current_key) + calculateAggregates(aggregate_columns, current_key_start, rows_with_current_key); + + block = header.cloneWithColumns(std::move(result_columns)); } void TTLBlockInputStream::calculateAggregates(const MutableColumns & aggregate_columns, size_t start_pos, size_t length) @@ -294,12 +303,14 @@ void TTLBlockInputStream::finalizeAggregates(MutableColumns & result_columns) { for (const auto & it : storage_rows_ttl.set_parts) it.expression->execute(agg_block); + for (const auto & name : storage_rows_ttl.group_by_keys) { const IColumn * values_column = agg_block.getByName(name).column.get(); auto & result_column = result_columns[header.getPositionByName(name)]; result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); } + for (const auto & it : storage_rows_ttl.set_parts) { const IColumn * values_column = agg_block.getByName(it.expression_result_column_name).column.get(); @@ -308,6 +319,7 @@ void TTLBlockInputStream::finalizeAggregates(MutableColumns & result_columns) } } } + agg_result.invalidate(); } diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/DataStreams/TTLBlockInputStream.h index 1d3b69f61c5..bbe1f8782a4 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/DataStreams/TTLBlockInputStream.h @@ -67,8 +67,13 @@ private: /// Removes values with expired ttl and computes new_ttl_infos and empty_columns for part void removeValuesWithExpiredColumnTTL(Block & block); + void executeRowsTTL(Block & block); + /// Removes rows with expired table ttl and computes new ttl_infos for part - void removeRowsWithExpiredTableTTL(Block & block); + void removeRowsWithExpiredTTL(Block & block, ColumnPtr ttl_column, ColumnPtr where_column); + + /// Aggregates rows with expired table ttl and computes new ttl_infos for part + void aggregateRowsWithExpiredTTL(Block & block, ColumnPtr ttl_column, ColumnPtr where_column); // Calculate aggregates of aggregate_columns into agg_result void calculateAggregates(const MutableColumns & aggregate_columns, size_t start_pos, size_t length); diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 7499f1de292..e412653a972 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -184,11 +184,8 @@ TTLDescription TTLDescription::getTTLFromAST( if (ttl_element->group_by_key.size() > pk_columns.size()) throw Exception("TTL Expression GROUP BY key should be a prefix of primary key", ErrorCodes::BAD_TTL_EXPRESSION); - NameSet primary_key_columns_set(pk_columns.begin(), pk_columns.end()); NameSet aggregation_columns_set; - - for (const auto & column : primary_key.expression->getRequiredColumns()) - primary_key_columns_set.insert(column); + NameSet used_primary_key_columns_set; for (size_t i = 0; i < ttl_element->group_by_key.size(); ++i) { @@ -196,6 +193,8 @@ TTLDescription TTLDescription::getTTLFromAST( throw Exception( "TTL Expression GROUP BY key should be a prefix of primary key", ErrorCodes::BAD_TTL_EXPRESSION); + + used_primary_key_columns_set.insert(pk_columns[i]); } for (const auto & [name, _] : ttl_element->group_by_aggregations) @@ -209,9 +208,17 @@ TTLDescription TTLDescription::getTTLFromAST( result.group_by_keys = Names(pk_columns.begin(), pk_columns.begin() + ttl_element->group_by_key.size()); auto aggregations = ttl_element->group_by_aggregations; + const auto & primary_key_expressions = primary_key.expression_list_ast->children; + for (size_t i = ttl_element->group_by_key.size(); i < primary_key_expressions.size(); ++i) + { + ASTPtr expr = makeASTFunction("any", primary_key_expressions[i]->clone()); + aggregations.emplace_back(pk_columns[i], std::move(expr)); + aggregation_columns_set.insert(pk_columns[i]); + } + for (const auto & column : columns.getOrdinary()) { - if (!aggregation_columns_set.count(column.name)) + if (!aggregation_columns_set.count(column.name) && !used_primary_key_columns_set.count(column.name)) { ASTPtr expr = makeASTFunction("any", std::make_shared(column.name)); aggregations.emplace_back(column.name, std::move(expr)); diff --git a/tests/queries/0_stateless/01280_ttl_where_group_by.reference b/tests/queries/0_stateless/01280_ttl_where_group_by.reference index ad20d38f2e6..7fe00709dee 100644 --- a/tests/queries/0_stateless/01280_ttl_where_group_by.reference +++ b/tests/queries/0_stateless/01280_ttl_where_group_by.reference @@ -1,20 +1,26 @@ +ttl_01280_1 1 1 0 4 1 2 3 7 1 3 0 5 2 1 0 1 2 1 20 1 +ttl_01280_2 1 1 [0,2,3] 4 1 1 [5,4,1] 13 1 3 [1,0,1,0] 17 2 1 [3,1,0,3] 8 3 1 [2,4,5] 8 +ttl_01280_3 1 1 0 4 -1 3 10 6 +1 1 10 6 2 1 0 3 -3 5 8 2 +3 1 8 2 +ttl_01280_4 1 1 0 4 -3 3 13 9 +10 2 13 9 +ttl_01280_5 1 2 7 5 2 3 6 5 -1 2 3 5 -2 3 3 5 +ttl_01280_6 +1 5 3 5 +2 10 3 5 diff --git a/tests/queries/0_stateless/01280_ttl_where_group_by.sh b/tests/queries/0_stateless/01280_ttl_where_group_by.sh index 9b05606f928..531f2951d36 100755 --- a/tests/queries/0_stateless/01280_ttl_where_group_by.sh +++ b/tests/queries/0_stateless/01280_ttl_where_group_by.sh @@ -13,6 +13,7 @@ function optimize() done } +echo "ttl_01280_1" $CLICKHOUSE_CLIENT -n --query " create table ttl_01280_1 (a Int, b Int, x Int, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second delete where x % 10 == 0 and y > 5; insert into ttl_01280_1 values (1, 1, 0, 4, now() + 10); @@ -29,6 +30,7 @@ $CLICKHOUSE_CLIENT --query "select a, b, x, y from ttl_01280_1 ORDER BY a, b, x, $CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_2" +echo "ttl_01280_2" $CLICKHOUSE_CLIENT -n --query " create table ttl_01280_2 (a Int, b Int, x Array(Int32), y Double, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a, b set x = minForEach(x), y = sum(y), d = max(d); insert into ttl_01280_2 values (1, 1, array(0, 2, 3), 4, now() + 10); @@ -47,6 +49,7 @@ $CLICKHOUSE_CLIENT --query "select a, b, x, y from ttl_01280_2 ORDER BY a, b, x, $CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_3" +echo "ttl_01280_3" $CLICKHOUSE_CLIENT -n --query " create table ttl_01280_3 (a Int, b Int, x Int64, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a set x = argMax(x, d), y = argMax(y, d), d = max(d); insert into ttl_01280_3 values (1, 1, 0, 4, now() + 10); @@ -65,6 +68,7 @@ $CLICKHOUSE_CLIENT --query "select a, b, x, y from ttl_01280_3 ORDER BY a, b, x, $CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_4" +echo "ttl_01280_4" $CLICKHOUSE_CLIENT -n --query " create table ttl_01280_4 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), -(a + b)) ttl d + interval 1 second group by toDate(d) set x = sum(x), y = max(y); insert into ttl_01280_4 values (1, 1, 0, 4, now() + 10); @@ -79,7 +83,8 @@ $CLICKHOUSE_CLIENT --query "select a, b, x, y from ttl_01280_4 ORDER BY a, b, x, $CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_5" -$CLICKHOUSE_CLIENT -n --query "create table ttl_01280_5 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), a, -b) ttl d + interval 1 second group by toDate(d), a set x = sum(x); +echo "ttl_01280_5" +$CLICKHOUSE_CLIENT -n --query "create table ttl_01280_5 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), a, -b) ttl d + interval 1 second group by toDate(d), a set x = sum(x), b = argMax(b, -b); insert into ttl_01280_5 values (1, 2, 3, 5, now()); insert into ttl_01280_5 values (2, 10, 1, 5, now()); insert into ttl_01280_5 values (2, 3, 5, 5, now()); @@ -91,6 +96,7 @@ $CLICKHOUSE_CLIENT --query "select a, b, x, y from ttl_01280_5 ORDER BY a, b, x, $CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_6" +echo "ttl_01280_6" $CLICKHOUSE_CLIENT -n --query " create table ttl_01280_6 (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), a, -b) ttl d + interval 1 second group by toDate(d), a; insert into ttl_01280_6 values (1, 2, 3, 5, now()); diff --git a/tests/queries/0_stateless/01280_ttl_where_group_by_negative.sql b/tests/queries/0_stateless/01280_ttl_where_group_by_negative.sql index f2c26a3d495..b273e065bcc 100644 --- a/tests/queries/0_stateless/01280_ttl_where_group_by_negative.sql +++ b/tests/queries/0_stateless/01280_ttl_where_group_by_negative.sql @@ -1,7 +1,4 @@ create table ttl_01280_error (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by x set y = max(y); -- { serverError 450} create table ttl_01280_error (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by b set y = max(y); -- { serverError 450} create table ttl_01280_error (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a, b, x set y = max(y); -- { serverError 450} -create table ttl_01280_error (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a set b = min(b), y = max(y); -- { serverError 450} create table ttl_01280_error (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a, b set y = max(y), y = max(y); -- { serverError 450} -create table ttl_01280_error (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (toDate(d), a) ttl d + interval 1 second group by toDate(d), a set d = min(d), b = max(b); -- { serverError 450} -create table ttl_01280_error (a Int, b Int, x Int64, y Int64, d DateTime) engine = MergeTree order by (d, -(a + b)) ttl d + interval 1 second group by d, -(a + b) set a = sum(a), b = min(b); -- { serverError 450} From b60c00ba7477ff4db5a9c9b7c962332c5248a4ce Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 25 Dec 2020 17:52:46 +0300 Subject: [PATCH 03/12] refactoring of TTL stream --- src/DataStreams/ITTLAlgorithm.cpp | 61 +++ src/DataStreams/ITTLAlgorithm.h | 43 ++ src/DataStreams/TTLAggregationAlgorithm.cpp | 173 +++++++ src/DataStreams/TTLAggregationAlgorithm.h | 40 ++ src/DataStreams/TTLBlockInputStream.cpp | 441 +++--------------- src/DataStreams/TTLBlockInputStream.h | 61 +-- src/DataStreams/TTLColumnAlgorithm.cpp | 88 ++++ src/DataStreams/TTLColumnAlgorithm.h | 29 ++ src/DataStreams/TTLDeleteAlgorithm.cpp | 58 +++ src/DataStreams/TTLDeleteAlgorithm.h | 21 + src/DataStreams/TTLUpdateInfoAlgorithm.cpp | 47 ++ src/DataStreams/TTLUpdateInfoAlgorithm.h | 31 ++ src/DataStreams/ya.make | 5 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 + .../MergeTree/MergeTreeDataPartTTLInfo.cpp | 87 ++-- .../MergeTree/MergeTreeDataPartTTLInfo.h | 5 +- .../MergeTree/MergeTreeDataWriter.cpp | 3 + src/Storages/StorageInMemoryMetadata.cpp | 12 +- src/Storages/StorageInMemoryMetadata.h | 4 + src/Storages/System/StorageSystemParts.cpp | 5 + src/Storages/TTLDescription.cpp | 8 +- src/Storages/TTLDescription.h | 2 + 22 files changed, 759 insertions(+), 471 deletions(-) create mode 100644 src/DataStreams/ITTLAlgorithm.cpp create mode 100644 src/DataStreams/ITTLAlgorithm.h create mode 100644 src/DataStreams/TTLAggregationAlgorithm.cpp create mode 100644 src/DataStreams/TTLAggregationAlgorithm.h create mode 100644 src/DataStreams/TTLColumnAlgorithm.cpp create mode 100644 src/DataStreams/TTLColumnAlgorithm.h create mode 100644 src/DataStreams/TTLDeleteAlgorithm.cpp create mode 100644 src/DataStreams/TTLDeleteAlgorithm.h create mode 100644 src/DataStreams/TTLUpdateInfoAlgorithm.cpp create mode 100644 src/DataStreams/TTLUpdateInfoAlgorithm.h diff --git a/src/DataStreams/ITTLAlgorithm.cpp b/src/DataStreams/ITTLAlgorithm.cpp new file mode 100644 index 00000000000..f0e98e9ab1c --- /dev/null +++ b/src/DataStreams/ITTLAlgorithm.cpp @@ -0,0 +1,61 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +ITTLAlgorithm::ITTLAlgorithm( + const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) + : description(description_) + , old_ttl_info(old_ttl_info_) + , current_time(current_time_) + , force(force_) + , date_lut(DateLUT::instance()) +{ +} + +bool ITTLAlgorithm::isTTLExpired(time_t ttl) const +{ + return (ttl && (ttl <= current_time)); +} + +ColumnPtr ITTLAlgorithm::extractRequieredColumn(const ExpressionActionsPtr & expression, const Block & block, const String & result_column) +{ + if (!expression) + return nullptr; + + if (block.has(result_column)) + return block.getByName(result_column).column; + + Block block_copy; + for (const auto & column_name : expression->getRequiredColumns()) + block_copy.insert(block.getByName(column_name)); + + expression->execute(block_copy); + return block_copy.getByName(result_column).column; +} + +UInt32 ITTLAlgorithm::getTimestampByIndex(const IColumn * column, size_t index) const +{ + if (const ColumnUInt16 * column_date = typeid_cast(column)) + return date_lut.fromDayNum(DayNum(column_date->getData()[index])); + else if (const ColumnUInt32 * column_date_time = typeid_cast(column)) + return column_date_time->getData()[index]; + else if (const ColumnConst * column_const = typeid_cast(column)) + { + if (typeid_cast(&column_const->getDataColumn())) + return date_lut.fromDayNum(DayNum(column_const->getValue())); + else if (typeid_cast(&column_const->getDataColumn())) + return column_const->getValue(); + } + + throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR); +} + +} diff --git a/src/DataStreams/ITTLAlgorithm.h b/src/DataStreams/ITTLAlgorithm.h new file mode 100644 index 00000000000..28a371e9289 --- /dev/null +++ b/src/DataStreams/ITTLAlgorithm.h @@ -0,0 +1,43 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class ITTLAlgorithm +{ +public: + using TTLInfo = IMergeTreeDataPart::TTLInfo; + using MutableDataPartPtr = MergeTreeMutableDataPartPtr; + + ITTLAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); + virtual ~ITTLAlgorithm() = default; + + virtual void execute(Block & block) = 0; + virtual void finalize(const MutableDataPartPtr & data_part) const = 0; + + bool isMinTTLExpired() const { return force || isTTLExpired(old_ttl_info.min); } + bool isMaxTTLExpired() const { return isTTLExpired(old_ttl_info.max); } + +protected: + bool isTTLExpired(time_t ttl) const; + UInt32 getTimestampByIndex(const IColumn * column, size_t index) const; + static ColumnPtr extractRequieredColumn(const ExpressionActionsPtr & expression, const Block & block, const String & result_column); + + const TTLDescription description; + const TTLInfo old_ttl_info; + const time_t current_time; + const bool force; + TTLInfo new_ttl_info; + +private: + const DateLUTImpl & date_lut; +}; + +using TTLAlgorithmPtr = std::unique_ptr; + +} diff --git a/src/DataStreams/TTLAggregationAlgorithm.cpp b/src/DataStreams/TTLAggregationAlgorithm.cpp new file mode 100644 index 00000000000..6cc1ac00b7e --- /dev/null +++ b/src/DataStreams/TTLAggregationAlgorithm.cpp @@ -0,0 +1,173 @@ +#include + +namespace DB +{ + +TTLAggregationAlgorithm::TTLAggregationAlgorithm( + const TTLDescription & description_, + const TTLInfo & old_ttl_info_, + time_t current_time_, + bool force_, + const Block & header_, + const MergeTreeData & storage_) + : ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_) + , header(header_) +{ + current_key_value.resize(description.group_by_keys.size()); + + ColumnNumbers keys; + for (const auto & key : description.group_by_keys) + keys.push_back(header.getPositionByName(key)); + + key_columns.resize(description.group_by_keys.size()); + AggregateDescriptions aggregates = description.aggregate_descriptions; + + for (auto & descr : aggregates) + if (descr.arguments.empty()) + for (const auto & name : descr.argument_names) + descr.arguments.push_back(header.getPositionByName(name)); + + columns_for_aggregator.resize(description.aggregate_descriptions.size()); + const Settings & settings = storage_.global_context.getSettingsRef(); + + Aggregator::Params params(header, keys, aggregates, + false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, 0, 0, + settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, + storage_.global_context.getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); + + aggregator = std::make_unique(params); +} + +void TTLAggregationAlgorithm::execute(Block & block) +{ + if (!block) + { + if (!aggregation_result.empty()) + { + MutableColumns result_columns = header.cloneEmptyColumns(); + finalizeAggregates(result_columns); + block = header.cloneWithColumns(std::move(result_columns)); + } + + return; + } + + const auto & column_names = header.getNames(); + MutableColumns result_columns = header.cloneEmptyColumns(); + MutableColumns aggregate_columns = header.cloneEmptyColumns(); + + auto ttl_column = extractRequieredColumn(description.expression, block, description.result_column); + auto where_column = extractRequieredColumn(description.where_expression, block, description.where_result_column); + + size_t rows_aggregated = 0; + size_t current_key_start = 0; + size_t rows_with_current_key = 0; + + for (size_t i = 0; i < block.rows(); ++i) + { + UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); + bool where_filter_passed = !where_column || where_column->getBool(i); + bool ttl_expired = isTTLExpired(cur_ttl) && where_filter_passed; + + bool same_as_current = true; + for (size_t j = 0; j < description.group_by_keys.size(); ++j) + { + const String & key_column = description.group_by_keys[j]; + const IColumn * values_column = block.getByName(key_column).column.get(); + if (!same_as_current || (*values_column)[i] != current_key_value[j]) + { + values_column->get(i, current_key_value[j]); + same_as_current = false; + } + } + + if (!same_as_current) + { + if (rows_with_current_key) + calculateAggregates(aggregate_columns, current_key_start, rows_with_current_key); + finalizeAggregates(result_columns); + + current_key_start = rows_aggregated; + rows_with_current_key = 0; + } + + if (ttl_expired) + { + ++rows_with_current_key; + ++rows_aggregated; + for (const auto & name : column_names) + { + const IColumn * values_column = block.getByName(name).column.get(); + auto & column = aggregate_columns[header.getPositionByName(name)]; + column->insertFrom(*values_column, i); + } + } + else + { + new_ttl_info.update(cur_ttl); + for (const auto & name : column_names) + { + const IColumn * values_column = block.getByName(name).column.get(); + auto & column = result_columns[header.getPositionByName(name)]; + column->insertFrom(*values_column, i); + } + } + } + + if (rows_with_current_key) + calculateAggregates(aggregate_columns, current_key_start, rows_with_current_key); + + block = header.cloneWithColumns(std::move(result_columns)); +} + +void TTLAggregationAlgorithm::calculateAggregates(const MutableColumns & aggregate_columns, size_t start_pos, size_t length) +{ + Columns aggregate_chunk; + aggregate_chunk.reserve(aggregate_columns.size()); + for (const auto & name : header.getNames()) + { + const auto & column = aggregate_columns[header.getPositionByName(name)]; + ColumnPtr chunk_column = column->cut(start_pos, length); + aggregate_chunk.emplace_back(std::move(chunk_column)); + } + + aggregator->executeOnBlock(aggregate_chunk, length, aggregation_result, key_columns, + columns_for_aggregator, no_more_keys); +} + +void TTLAggregationAlgorithm::finalizeAggregates(MutableColumns & result_columns) +{ + if (!aggregation_result.empty()) + { + auto aggregated_res = aggregator->convertToBlocks(aggregation_result, true, 1); + for (auto & agg_block : aggregated_res) + { + for (const auto & it : description.set_parts) + it.expression->execute(agg_block); + + for (const auto & name : description.group_by_keys) + { + const IColumn * values_column = agg_block.getByName(name).column.get(); + auto & result_column = result_columns[header.getPositionByName(name)]; + result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); + } + + for (const auto & it : description.set_parts) + { + const IColumn * values_column = agg_block.getByName(it.expression_result_column_name).column.get(); + auto & result_column = result_columns[header.getPositionByName(it.column_name)]; + result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); + } + } + } + + aggregation_result.invalidate(); +} + +void TTLAggregationAlgorithm::finalize(const MutableDataPartPtr & data_part) const +{ + data_part->ttl_infos.group_by_ttl[description.result_column] = new_ttl_info; + data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); +} + +} diff --git a/src/DataStreams/TTLAggregationAlgorithm.h b/src/DataStreams/TTLAggregationAlgorithm.h new file mode 100644 index 00000000000..977e755ca8b --- /dev/null +++ b/src/DataStreams/TTLAggregationAlgorithm.h @@ -0,0 +1,40 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class TTLAggregationAlgorithm final : public ITTLAlgorithm +{ +public: + TTLAggregationAlgorithm( + const TTLDescription & description_, + const TTLInfo & old_ttl_info_, + time_t current_time_, + bool force_, + const Block & header_, + const MergeTreeData & storage_); + + void execute(Block & block) override; + void finalize(const MutableDataPartPtr & data_part) const override; + +private: + // Calculate aggregates of aggregate_columns into aggregation_result + void calculateAggregates(const MutableColumns & aggregate_columns, size_t start_pos, size_t length); + + /// Finalize aggregation_result into result_columns + void finalizeAggregates(MutableColumns & result_columns); + + const Block header; + std::unique_ptr aggregator; + Row current_key_value; + AggregatedDataVariants aggregation_result; + ColumnRawPtrs key_columns; + Aggregator::AggregateColumns columns_for_aggregator; + bool no_more_keys = false; +}; + +} diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index ab80f69d4d2..8c680f6875b 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -8,15 +8,14 @@ #include #include +#include +#include +#include +#include + namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - TTLBlockInputStream::TTLBlockInputStream( const BlockInputStreamPtr & input_, const MergeTreeData & storage_, @@ -24,83 +23,62 @@ TTLBlockInputStream::TTLBlockInputStream( const MergeTreeData::MutableDataPartPtr & data_part_, time_t current_time_, bool force_) - : storage(storage_) - , metadata_snapshot(metadata_snapshot_) - , data_part(data_part_) - , current_time(current_time_) - , force(force_) - , old_ttl_infos(data_part->ttl_infos) - , log(&Poco::Logger::get(storage.getLogName() + " (TTLBlockInputStream)")) - , date_lut(DateLUT::instance()) + : data_part(data_part_) + , log(&Poco::Logger::get(storage_.getLogName() + " (TTLBlockInputStream)")) { children.push_back(input_); header = children.at(0)->getHeader(); + auto old_ttl_infos = data_part->ttl_infos; - const auto & storage_columns = metadata_snapshot->getColumns(); - const auto & column_defaults = storage_columns.getDefaults(); - - ASTPtr default_expr_list = std::make_shared(); - for (const auto & [name, _] : metadata_snapshot->getColumnTTLs()) + if (metadata_snapshot_->hasRowsTTL()) { - auto it = column_defaults.find(name); - if (it != column_defaults.end()) + const auto & rows_ttl = metadata_snapshot_->getRowsTTL(); + auto algorithm = std::make_unique( + rows_ttl, old_ttl_infos.table_ttl, current_time_, force_); + + /// Skip all data if table ttl is expired for part + if (algorithm->isMaxTTLExpired() && !rows_ttl.where_expression) + all_data_dropped = true; + + delete_algorithm = algorithm.get(); + algorithms.emplace_back(std::move(algorithm)); + } + + for (const auto & group_by_ttl : metadata_snapshot_->getGroupByTTLs()) + algorithms.emplace_back(std::make_unique( + group_by_ttl, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_, header, storage_)); + + if (metadata_snapshot_->hasAnyColumnTTL()) + { + const auto & storage_columns = metadata_snapshot_->getColumns(); + const auto & column_defaults = storage_columns.getDefaults(); + + for (const auto & [name, description] : metadata_snapshot_->getColumnTTLs()) { - auto column = storage_columns.get(name); - auto expression = it->second.expression->clone(); - default_expr_list->children.emplace_back(setAlias(addTypeConversionToAST(std::move(expression), column.type->getName()), it->first)); + ExpressionActionsPtr default_expression; + auto it = column_defaults.find(name); + if (it != column_defaults.end()) + { + const auto & column = storage_columns.get(name); + auto default_ast = it->second.expression->clone(); + default_ast = setAlias(addTypeConversionToAST(std::move(default_ast), column.type->getName()), it->first); + + auto syntax_result = TreeRewriter(storage_.global_context).analyze(default_ast, metadata_snapshot_->getColumns().getAllPhysical()); + default_expression = ExpressionAnalyzer{default_ast, syntax_result, storage_.global_context}.getActions(true); + } + + algorithms.emplace_back(std::make_unique( + description, old_ttl_infos.columns_ttl[name], current_time_, force_, name, default_expression)); } } - for (const auto & [name, ttl_info] : old_ttl_infos.columns_ttl) - { - if (force || isTTLExpired(ttl_info.min)) - { - new_ttl_infos.columns_ttl.emplace(name, IMergeTreeDataPart::TTLInfo{}); - empty_columns.emplace(name); - } - else - new_ttl_infos.columns_ttl.emplace(name, ttl_info); - } + for (const auto & move_ttl : metadata_snapshot_->getMoveTTLs()) + algorithms.emplace_back(std::make_unique( + move_ttl, old_ttl_infos.moves_ttl[move_ttl.result_column], current_time_, force_)); - if (!force && !isTTLExpired(old_ttl_infos.table_ttl.min)) - new_ttl_infos.table_ttl = old_ttl_infos.table_ttl; - - if (!default_expr_list->children.empty()) - { - auto syntax_result = TreeRewriter(storage.global_context).analyze(default_expr_list, metadata_snapshot->getColumns().getAllPhysical()); - defaults_expression = ExpressionAnalyzer{default_expr_list, syntax_result, storage.global_context}.getActions(true); - } - - auto storage_rows_ttl = metadata_snapshot->getRowsTTL(); - if (metadata_snapshot->hasRowsTTL() && storage_rows_ttl.mode == TTLMode::GROUP_BY) - { - current_key_value.resize(storage_rows_ttl.group_by_keys.size()); - - ColumnNumbers keys; - for (const auto & key : storage_rows_ttl.group_by_keys) - keys.push_back(header.getPositionByName(key)); - agg_key_columns.resize(storage_rows_ttl.group_by_keys.size()); - - AggregateDescriptions aggregates = storage_rows_ttl.aggregate_descriptions; - for (auto & descr : aggregates) - if (descr.arguments.empty()) - for (const auto & name : descr.argument_names) - descr.arguments.push_back(header.getPositionByName(name)); - - agg_aggregate_columns.resize(storage_rows_ttl.aggregate_descriptions.size()); - const Settings & settings = storage.global_context.getSettingsRef(); - - Aggregator::Params params(header, keys, aggregates, - false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, 0, 0, - settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, - storage.global_context.getTemporaryVolume(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); - aggregator = std::make_unique(params); - } -} - -bool TTLBlockInputStream::isTTLExpired(time_t ttl) const -{ - return (ttl && (ttl <= current_time)); + for (const auto & recompression_ttl : metadata_snapshot_->getRecompressionTTLs()) + algorithms.emplace_back(std::make_unique( + recompression_ttl, old_ttl_infos.recompression_ttl[recompression_ttl.result_column], current_time_, force_)); } Block reorderColumns(Block block, const Block & header) @@ -114,325 +92,30 @@ Block reorderColumns(Block block, const Block & header) Block TTLBlockInputStream::readImpl() { - /// Skip all data if table ttl is expired for part - auto storage_rows_ttl = metadata_snapshot->getRowsTTL(); - if (metadata_snapshot->hasRowsTTL() - && !storage_rows_ttl.where_expression - && storage_rows_ttl.mode != TTLMode::GROUP_BY - && isTTLExpired(old_ttl_infos.table_ttl.max)) - { - rows_removed = data_part->rows_count; + if (all_data_dropped) return {}; - } - Block block = children.at(0)->read(); + auto block = children.at(0)->read(); + for (const auto & algorithm : algorithms) + algorithm->execute(block); + if (!block) - { - if (aggregator && !agg_result.empty()) - { - MutableColumns result_columns = header.cloneEmptyColumns(); - finalizeAggregates(result_columns); - block = header.cloneWithColumns(std::move(result_columns)); - } - return block; - } - - if (metadata_snapshot->hasRowsTTL() && (force || isTTLExpired(old_ttl_infos.table_ttl.min))) - executeRowsTTL(block); - - removeValuesWithExpiredColumnTTL(block); - updateMovesTTL(block); - updateRecompressionTTL(block); return reorderColumns(std::move(block), header); } void TTLBlockInputStream::readSuffixImpl() { - for (const auto & elem : new_ttl_infos.columns_ttl) - new_ttl_infos.updatePartMinMaxTTL(elem.second.min, elem.second.max); + data_part->ttl_infos = {}; + for (const auto & algorithm : algorithms) + algorithm->finalize(data_part); - new_ttl_infos.updatePartMinMaxTTL(new_ttl_infos.table_ttl.min, new_ttl_infos.table_ttl.max); - - data_part->ttl_infos = std::move(new_ttl_infos); - data_part->expired_columns = std::move(empty_columns); - - if (rows_removed) + if (delete_algorithm) + { + size_t rows_removed = all_data_dropped ? data_part->rows_count : delete_algorithm->getNumberOfRemovedRows(); LOG_DEBUG(log, "Removed {} rows with expired TTL from part {}", rows_removed, data_part->name); -} - -static ColumnPtr extractRequieredColumn(const ExpressionActions & expression, const Block & block, const String & result_column) -{ - if (block.has(result_column)) - return block.getByName(result_column).column; - - Block block_copy; - for (const auto & column_name : expression.getRequiredColumns()) - block_copy.insert(block.getByName(column_name)); - - expression.execute(block_copy); - return block_copy.getByName(result_column).column; -} - -void TTLBlockInputStream::executeRowsTTL(Block & block) -{ - auto rows_ttl = metadata_snapshot->getRowsTTL(); - auto ttl_column = extractRequieredColumn(*rows_ttl.expression, block, rows_ttl.result_column); - - auto where_result_column = rows_ttl.where_expression ? - extractRequieredColumn(*rows_ttl.where_expression, block, rows_ttl.where_result_column): nullptr; - - if (aggregator) - aggregateRowsWithExpiredTTL(block, ttl_column, where_result_column); - else - removeRowsWithExpiredTTL(block, ttl_column, where_result_column); -} - -void TTLBlockInputStream::removeRowsWithExpiredTTL(Block & block, ColumnPtr ttl_column, ColumnPtr where_column) -{ - MutableColumns result_columns; - const auto & column_names = header.getNames(); - - result_columns.reserve(column_names.size()); - for (auto it = column_names.begin(); it != column_names.end(); ++it) - { - const IColumn * values_column = block.getByName(*it).column.get(); - MutableColumnPtr result_column = values_column->cloneEmpty(); - result_column->reserve(block.rows()); - - for (size_t i = 0; i < block.rows(); ++i) - { - UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); - bool where_filter_passed = !where_column || where_column->getBool(i); - if (!isTTLExpired(cur_ttl) || !where_filter_passed) - { - new_ttl_infos.table_ttl.update(cur_ttl); - result_column->insertFrom(*values_column, i); - } - else if (it == column_names.begin()) - ++rows_removed; - } - - result_columns.emplace_back(std::move(result_column)); - } - - block = header.cloneWithColumns(std::move(result_columns)); -} - -void TTLBlockInputStream::aggregateRowsWithExpiredTTL(Block & block, ColumnPtr ttl_column, ColumnPtr where_column) -{ - const auto & column_names = header.getNames(); - MutableColumns result_columns = header.cloneEmptyColumns(); - MutableColumns aggregate_columns = header.cloneEmptyColumns(); - - size_t rows_aggregated = 0; - size_t current_key_start = 0; - size_t rows_with_current_key = 0; - auto storage_rows_ttl = metadata_snapshot->getRowsTTL(); - - for (size_t i = 0; i < block.rows(); ++i) - { - UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); - bool where_filter_passed = !where_column || where_column->getBool(i); - bool ttl_expired = isTTLExpired(cur_ttl) && where_filter_passed; - - bool same_as_current = true; - for (size_t j = 0; j < storage_rows_ttl.group_by_keys.size(); ++j) - { - const String & key_column = storage_rows_ttl.group_by_keys[j]; - const IColumn * values_column = block.getByName(key_column).column.get(); - if (!same_as_current || (*values_column)[i] != current_key_value[j]) - { - values_column->get(i, current_key_value[j]); - same_as_current = false; - } - } - - if (!same_as_current) - { - if (rows_with_current_key) - calculateAggregates(aggregate_columns, current_key_start, rows_with_current_key); - finalizeAggregates(result_columns); - - current_key_start = rows_aggregated; - rows_with_current_key = 0; - } - - if (ttl_expired) - { - ++rows_with_current_key; - ++rows_aggregated; - for (const auto & name : column_names) - { - const IColumn * values_column = block.getByName(name).column.get(); - auto & column = aggregate_columns[header.getPositionByName(name)]; - column->insertFrom(*values_column, i); - } - } - else - { - new_ttl_infos.table_ttl.update(cur_ttl); - for (const auto & name : column_names) - { - const IColumn * values_column = block.getByName(name).column.get(); - auto & column = result_columns[header.getPositionByName(name)]; - column->insertFrom(*values_column, i); - } - } - } - - if (rows_with_current_key) - calculateAggregates(aggregate_columns, current_key_start, rows_with_current_key); - - block = header.cloneWithColumns(std::move(result_columns)); -} - -void TTLBlockInputStream::calculateAggregates(const MutableColumns & aggregate_columns, size_t start_pos, size_t length) -{ - Columns aggregate_chunk; - aggregate_chunk.reserve(aggregate_columns.size()); - for (const auto & name : header.getNames()) - { - const auto & column = aggregate_columns[header.getPositionByName(name)]; - ColumnPtr chunk_column = column->cut(start_pos, length); - aggregate_chunk.emplace_back(std::move(chunk_column)); - } - aggregator->executeOnBlock(aggregate_chunk, length, agg_result, agg_key_columns, - agg_aggregate_columns, agg_no_more_keys); -} - -void TTLBlockInputStream::finalizeAggregates(MutableColumns & result_columns) -{ - if (!agg_result.empty()) - { - auto aggregated_res = aggregator->convertToBlocks(agg_result, true, 1); - auto storage_rows_ttl = metadata_snapshot->getRowsTTL(); - for (auto & agg_block : aggregated_res) - { - for (const auto & it : storage_rows_ttl.set_parts) - it.expression->execute(agg_block); - - for (const auto & name : storage_rows_ttl.group_by_keys) - { - const IColumn * values_column = agg_block.getByName(name).column.get(); - auto & result_column = result_columns[header.getPositionByName(name)]; - result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); - } - - for (const auto & it : storage_rows_ttl.set_parts) - { - const IColumn * values_column = agg_block.getByName(it.expression_result_column_name).column.get(); - auto & result_column = result_columns[header.getPositionByName(it.column_name)]; - result_column->insertRangeFrom(*values_column, 0, agg_block.rows()); - } - } - } - - agg_result.invalidate(); -} - -void TTLBlockInputStream::removeValuesWithExpiredColumnTTL(Block & block) -{ - Block block_with_defaults; - if (defaults_expression) - { - block_with_defaults = block; - defaults_expression->execute(block_with_defaults); - } - - for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs()) - { - /// If we read not all table columns. E.g. while mutation. - if (!block.has(name)) - continue; - - const auto & old_ttl_info = old_ttl_infos.columns_ttl[name]; - auto & new_ttl_info = new_ttl_infos.columns_ttl[name]; - - /// Nothing to do - if (!force && !isTTLExpired(old_ttl_info.min)) - continue; - - /// Later drop full column - if (isTTLExpired(old_ttl_info.max)) - continue; - - auto ttl_column = extractRequieredColumn(*ttl_entry.expression, block, ttl_entry.result_column); - - ColumnPtr default_column = nullptr; - if (block_with_defaults.has(name)) - default_column = block_with_defaults.getByName(name).column->convertToFullColumnIfConst(); - - auto & column_with_type = block.getByName(name); - const IColumn * values_column = column_with_type.column.get(); - MutableColumnPtr result_column = values_column->cloneEmpty(); - result_column->reserve(block.rows()); - - for (size_t i = 0; i < block.rows(); ++i) - { - UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); - if (isTTLExpired(cur_ttl)) - { - if (default_column) - result_column->insertFrom(*default_column, i); - else - result_column->insertDefault(); - } - else - { - new_ttl_info.update(cur_ttl); - empty_columns.erase(name); - result_column->insertFrom(*values_column, i); - } - } - column_with_type.column = std::move(result_column); } } -void TTLBlockInputStream::updateTTLWithDescriptions(Block & block, const TTLDescriptions & descriptions, TTLInfoMap & ttl_info_map) -{ - for (const auto & ttl_entry : descriptions) - { - auto & new_ttl_info = ttl_info_map[ttl_entry.result_column]; - if (!block.has(ttl_entry.result_column)) - ttl_entry.expression->execute(block); - - auto ttl_column = extractRequieredColumn(*ttl_entry.expression, block, ttl_entry.result_column); - - for (size_t i = 0; i < block.rows(); ++i) - { - UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); - new_ttl_info.update(cur_ttl); - } - } -} - -void TTLBlockInputStream::updateMovesTTL(Block & block) -{ - updateTTLWithDescriptions(block, metadata_snapshot->getMoveTTLs(), new_ttl_infos.moves_ttl); -} - -void TTLBlockInputStream::updateRecompressionTTL(Block & block) -{ - updateTTLWithDescriptions(block, metadata_snapshot->getRecompressionTTLs(), new_ttl_infos.recompression_ttl); -} - -UInt32 TTLBlockInputStream::getTimestampByIndex(const IColumn * column, size_t ind) -{ - if (const ColumnUInt16 * column_date = typeid_cast(column)) - return date_lut.fromDayNum(DayNum(column_date->getData()[ind])); - else if (const ColumnUInt32 * column_date_time = typeid_cast(column)) - return column_date_time->getData()[ind]; - else if (const ColumnConst * column_const = typeid_cast(column)) - { - if (typeid_cast(&column_const->getDataColumn())) - return date_lut.fromDayNum(DayNum(column_const->getValue())); - else if (typeid_cast(&column_const->getDataColumn())) - return column_const->getValue(); - } - - throw Exception("Unexpected type of result TTL column", ErrorCodes::LOGICAL_ERROR); -} - } diff --git a/src/DataStreams/TTLBlockInputStream.h b/src/DataStreams/TTLBlockInputStream.h index bbe1f8782a4..da86b8d5710 100644 --- a/src/DataStreams/TTLBlockInputStream.h +++ b/src/DataStreams/TTLBlockInputStream.h @@ -3,8 +3,9 @@ #include #include #include -#include #include +#include +#include #include @@ -24,7 +25,6 @@ public: ); String getName() const override { return "TTL"; } - Block getHeader() const override { return header; } protected: @@ -34,65 +34,14 @@ protected: void readSuffixImpl() override; private: - const MergeTreeData & storage; - StorageMetadataPtr metadata_snapshot; + std::vector algorithms; + const TTLDeleteAlgorithm * delete_algorithm = nullptr; + bool all_data_dropped = false; /// ttl_infos and empty_columns are updating while reading const MergeTreeData::MutableDataPartPtr & data_part; - - time_t current_time; - bool force; - - std::unique_ptr aggregator; - std::vector current_key_value; - AggregatedDataVariants agg_result; - ColumnRawPtrs agg_key_columns; - Aggregator::AggregateColumns agg_aggregate_columns; - bool agg_no_more_keys = false; - - IMergeTreeDataPart::TTLInfos old_ttl_infos; - IMergeTreeDataPart::TTLInfos new_ttl_infos; - NameSet empty_columns; - - size_t rows_removed = 0; Poco::Logger * log; - const DateLUTImpl & date_lut; - - /// TODO rewrite defaults logic to evaluteMissingDefaults - std::unordered_map defaults_result_column; - ExpressionActionsPtr defaults_expression; - Block header; -private: - /// Removes values with expired ttl and computes new_ttl_infos and empty_columns for part - void removeValuesWithExpiredColumnTTL(Block & block); - - void executeRowsTTL(Block & block); - - /// Removes rows with expired table ttl and computes new ttl_infos for part - void removeRowsWithExpiredTTL(Block & block, ColumnPtr ttl_column, ColumnPtr where_column); - - /// Aggregates rows with expired table ttl and computes new ttl_infos for part - void aggregateRowsWithExpiredTTL(Block & block, ColumnPtr ttl_column, ColumnPtr where_column); - - // Calculate aggregates of aggregate_columns into agg_result - void calculateAggregates(const MutableColumns & aggregate_columns, size_t start_pos, size_t length); - - /// Finalize agg_result into result_columns - void finalizeAggregates(MutableColumns & result_columns); - - /// Execute description expressions on block and update ttl's in - /// ttl_info_map with expression results. - void updateTTLWithDescriptions(Block & block, const TTLDescriptions & descriptions, TTLInfoMap & ttl_info_map); - - /// Updates TTL for moves - void updateMovesTTL(Block & block); - - /// Update values for recompression TTL using data from block. - void updateRecompressionTTL(Block & block); - - UInt32 getTimestampByIndex(const IColumn * column, size_t ind); - bool isTTLExpired(time_t ttl) const; }; } diff --git a/src/DataStreams/TTLColumnAlgorithm.cpp b/src/DataStreams/TTLColumnAlgorithm.cpp new file mode 100644 index 00000000000..4747a605e3b --- /dev/null +++ b/src/DataStreams/TTLColumnAlgorithm.cpp @@ -0,0 +1,88 @@ +#include + +namespace DB +{ + +TTLColumnAlgorithm::TTLColumnAlgorithm( + const TTLDescription & description_, + const TTLInfo & old_ttl_info_, + time_t current_time_, + bool force_, + const String & column_name_, + const ExpressionActionsPtr & default_expression_) + : ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_) + , column_name(column_name_) + , default_expression(default_expression_) +{ + if (!isMinTTLExpired()) + { + new_ttl_info = old_ttl_info; + is_fully_empty = false; + } +} + +void TTLColumnAlgorithm::execute(Block & block) +{ + if (!block) + return; + + + /// If we read not all table columns. E.g. while mutation. + if (!block.has(column_name)) + return; + + /// Nothing to do + if (!isMinTTLExpired()) + return; + + /// Later drop full column + if (isMaxTTLExpired()) + return; + + //// TODO: use extractRequiredColumn + ColumnPtr default_column; + if (default_expression) + { + Block block_with_defaults; + block_with_defaults = block; + default_expression->execute(block_with_defaults); + default_column = block_with_defaults.getByName(column_name).column->convertToFullColumnIfConst(); + } + + auto ttl_column = extractRequieredColumn(description.expression, block, description.result_column); + + auto & column_with_type = block.getByName(column_name); + const IColumn * values_column = column_with_type.column.get(); + MutableColumnPtr result_column = values_column->cloneEmpty(); + result_column->reserve(block.rows()); + + for (size_t i = 0; i < block.rows(); ++i) + { + UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); + if (isTTLExpired(cur_ttl)) + { + if (default_column) + result_column->insertFrom(*default_column, i); + else + result_column->insertDefault(); + } + else + { + new_ttl_info.update(cur_ttl); + is_fully_empty = false; + result_column->insertFrom(*values_column, i); + } + } + + column_with_type.column = std::move(result_column); +} + +void TTLColumnAlgorithm::finalize(const MutableDataPartPtr & data_part) const +{ + data_part->ttl_infos.columns_ttl[column_name] = new_ttl_info; + data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); + if (is_fully_empty) + data_part->expired_columns.insert(column_name); +} + +} diff --git a/src/DataStreams/TTLColumnAlgorithm.h b/src/DataStreams/TTLColumnAlgorithm.h new file mode 100644 index 00000000000..b2824dba9b0 --- /dev/null +++ b/src/DataStreams/TTLColumnAlgorithm.h @@ -0,0 +1,29 @@ +#pragma once + +#include + +namespace DB +{ + +class TTLColumnAlgorithm final : public ITTLAlgorithm +{ +public: + TTLColumnAlgorithm( + const TTLDescription & description_, + const TTLInfo & old_ttl_info_, + time_t current_time_, + bool force_, + const String & column_name_, + const ExpressionActionsPtr & default_expression_); + + void execute(Block & block) override; + void finalize(const MutableDataPartPtr & data_part) const override; + +private: + const String column_name; + const ExpressionActionsPtr default_expression; + + bool is_fully_empty = true; +}; + +} diff --git a/src/DataStreams/TTLDeleteAlgorithm.cpp b/src/DataStreams/TTLDeleteAlgorithm.cpp new file mode 100644 index 00000000000..9ff4eb767df --- /dev/null +++ b/src/DataStreams/TTLDeleteAlgorithm.cpp @@ -0,0 +1,58 @@ +#include + +namespace DB +{ + +TTLDeleteAlgorithm::TTLDeleteAlgorithm( + const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) + : ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_) +{ + if (!isMinTTLExpired()) + new_ttl_info = old_ttl_info; +} + +void TTLDeleteAlgorithm::execute(Block & block) +{ + if (!block || !isMinTTLExpired()) + return; + + auto ttl_column = extractRequieredColumn(description.expression, block, description.result_column); + auto where_column = extractRequieredColumn(description.where_expression, block, description.where_result_column); + + MutableColumns result_columns; + const auto & column_names = block.getNames(); + + result_columns.reserve(column_names.size()); + for (auto it = column_names.begin(); it != column_names.end(); ++it) + { + const IColumn * values_column = block.getByName(*it).column.get(); + MutableColumnPtr result_column = values_column->cloneEmpty(); + result_column->reserve(block.rows()); + + for (size_t i = 0; i < block.rows(); ++i) + { + UInt32 cur_ttl = getTimestampByIndex(ttl_column.get(), i); + bool where_filter_passed = !where_column || where_column->getBool(i); + + if (!isTTLExpired(cur_ttl) || !where_filter_passed) + { + new_ttl_info.update(cur_ttl); + result_column->insertFrom(*values_column, i); + } + else if (it == column_names.begin()) + ++rows_removed; + } + + result_columns.emplace_back(std::move(result_column)); + } + + block = block.cloneWithColumns(std::move(result_columns)); +} + +void TTLDeleteAlgorithm::finalize(const MutableDataPartPtr & data_part) const +{ + data_part->ttl_infos.table_ttl = new_ttl_info; + data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); +} + +} diff --git a/src/DataStreams/TTLDeleteAlgorithm.h b/src/DataStreams/TTLDeleteAlgorithm.h new file mode 100644 index 00000000000..36da59da46e --- /dev/null +++ b/src/DataStreams/TTLDeleteAlgorithm.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + +class TTLDeleteAlgorithm final : public ITTLAlgorithm +{ +public: + TTLDeleteAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); + + void execute(Block & block) override; + void finalize(const MutableDataPartPtr & data_part) const override; + size_t getNumberOfRemovedRows() const { return rows_removed; } + +private: + size_t rows_removed = 0; +}; + +} diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp b/src/DataStreams/TTLUpdateInfoAlgorithm.cpp new file mode 100644 index 00000000000..ce4d4128eec --- /dev/null +++ b/src/DataStreams/TTLUpdateInfoAlgorithm.cpp @@ -0,0 +1,47 @@ +#include + +namespace DB +{ + +TTLUpdateInfoAlgorithm::TTLUpdateInfoAlgorithm( + const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) + : ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_) +{ +} + +void TTLUpdateInfoAlgorithm::execute(Block & block) +{ + if (!block) + return; + + auto ttl_column = extractRequieredColumn(description.expression, block, description.result_column); + for (size_t i = 0; i < block.rows(); ++i) + { + UInt32 cur_ttl = ITTLAlgorithm::getTimestampByIndex(ttl_column.get(), i); + new_ttl_info.update(cur_ttl); + } +} + +TTLMoveAlgorithm::TTLMoveAlgorithm( + const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) + : TTLUpdateInfoAlgorithm(description_, old_ttl_info_, current_time_, force_) +{ +} + +void TTLMoveAlgorithm::finalize(const MutableDataPartPtr & data_part) const +{ + data_part->ttl_infos.moves_ttl[description.result_column] = new_ttl_info; +} + +TTLRecompressionAlgorithm::TTLRecompressionAlgorithm( + const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_) + : TTLUpdateInfoAlgorithm(description_, old_ttl_info_, current_time_, force_) +{ +} + +void TTLRecompressionAlgorithm::finalize(const MutableDataPartPtr & data_part) const +{ + data_part->ttl_infos.recompression_ttl[description.result_column] = new_ttl_info; +} + +} diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.h b/src/DataStreams/TTLUpdateInfoAlgorithm.h new file mode 100644 index 00000000000..4a680c5bb3a --- /dev/null +++ b/src/DataStreams/TTLUpdateInfoAlgorithm.h @@ -0,0 +1,31 @@ +#pragma once + +#include + +namespace DB +{ + +class TTLUpdateInfoAlgorithm : public ITTLAlgorithm +{ +public: + TTLUpdateInfoAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); + + void execute(Block & block) override; + void finalize(const MutableDataPartPtr & data_part) const override = 0; +}; + +class TTLMoveAlgorithm final : public TTLUpdateInfoAlgorithm +{ +public: + TTLMoveAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); + void finalize(const MutableDataPartPtr & data_part) const override; +}; + +class TTLRecompressionAlgorithm final : public TTLUpdateInfoAlgorithm +{ +public: + TTLRecompressionAlgorithm(const TTLDescription & description_, const TTLInfo & old_ttl_info_, time_t current_time_, bool force_); + void finalize(const MutableDataPartPtr & data_part) const override; +}; + +} diff --git a/src/DataStreams/ya.make b/src/DataStreams/ya.make index 858bf7081e7..b0a7755c7f9 100644 --- a/src/DataStreams/ya.make +++ b/src/DataStreams/ya.make @@ -27,6 +27,7 @@ SRCS( ExecutionSpeedLimits.cpp ExpressionBlockInputStream.cpp IBlockInputStream.cpp + ITTLAlgorithm.cpp InputStreamFromASTInsertQuery.cpp InternalTextLogsRowOutputStream.cpp LimitBlockInputStream.cpp @@ -44,7 +45,11 @@ SRCS( SquashingBlockInputStream.cpp SquashingBlockOutputStream.cpp SquashingTransform.cpp + TTLAggregationAlgorithm.cpp TTLBlockInputStream.cpp + TTLColumnAlgorithm.cpp + TTLDeleteAlgorithm.cpp + TTLUpdateInfoAlgorithm.cpp copyData.cpp finalizeBlock.cpp materializeBlock.cpp diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 104eedf060e..cdf66ec43f6 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1136,6 +1136,12 @@ bool IMergeTreeDataPart::checkAllTTLCalculated(const StorageMetadataPtr & metada return false; } + for (const auto & group_by_desc : metadata_snapshot->getGroupByTTLs()) + { + if (!ttl_infos.group_by_ttl.count(group_by_desc.result_column)) + return false; + } + return true; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp index 92c8a66e828..3a0bb283b63 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp @@ -17,13 +17,17 @@ void MergeTreeDataPartTTLInfos::update(const MergeTreeDataPartTTLInfos & other_i updatePartMinMaxTTL(ttl_info.min, ttl_info.max); } + for (const auto & [name, ttl_info] : other_infos.group_by_ttl) + { + group_by_ttl[name].update(ttl_info); + updatePartMinMaxTTL(ttl_info.min, ttl_info.max); + } + for (const auto & [name, ttl_info] : other_infos.recompression_ttl) recompression_ttl[name].update(ttl_info); for (const auto & [expression, ttl_info] : other_infos.moves_ttl) - { moves_ttl[expression].update(ttl_info); - } table_ttl.update(other_infos.table_ttl); updatePartMinMaxTTL(table_ttl.min, table_ttl.max); @@ -59,29 +63,33 @@ void MergeTreeDataPartTTLInfos::read(ReadBuffer & in) updatePartMinMaxTTL(table_ttl.min, table_ttl.max); } + + auto fill_ttl_info_map = [](const JSON & json_part, TTLInfoMap & ttl_info_map) + { + for (auto elem : json_part) // NOLINT + { + MergeTreeDataPartTTLInfo ttl_info; + ttl_info.min = elem["min"].getUInt(); + ttl_info.max = elem["max"].getUInt(); + String expression = elem["expression"].getString(); + ttl_info_map.emplace(expression, ttl_info); + } + }; + if (json.has("moves")) { const JSON & moves = json["moves"]; - for (auto move : moves) // NOLINT - { - MergeTreeDataPartTTLInfo ttl_info; - ttl_info.min = move["min"].getUInt(); - ttl_info.max = move["max"].getUInt(); - String expression = move["expression"].getString(); - moves_ttl.emplace(expression, ttl_info); - } + fill_ttl_info_map(moves, moves_ttl); } if (json.has("recompression")) { const JSON & recompressions = json["recompression"]; - for (auto recompression : recompressions) // NOLINT - { - MergeTreeDataPartTTLInfo ttl_info; - ttl_info.min = recompression["min"].getUInt(); - ttl_info.max = recompression["max"].getUInt(); - String expression = recompression["expression"].getString(); - recompression_ttl.emplace(expression, ttl_info); - } + fill_ttl_info_map(recompressions, recompression_ttl); + } + if (json.has("group_by")) + { + const JSON & group_by = json["group_by"]; + fill_ttl_info_map(group_by, group_by_ttl); } } @@ -118,6 +126,18 @@ void MergeTreeDataPartTTLInfos::write(WriteBuffer & out) const writeIntText(table_ttl.max, out); writeString("}", out); } + + auto write_info_for_expression = [&out](const auto & name, const auto & info) + { + writeString(R"({"expression":)", out); + writeString(doubleQuoteString(name), out); + writeString(R"(,"min":)", out); + writeIntText(info.min, out); + writeString(R"(,"max":)", out); + writeIntText(info.max, out); + writeString("}", out); + }; + if (!moves_ttl.empty()) { if (!columns_ttl.empty() || table_ttl.min) @@ -128,13 +148,7 @@ void MergeTreeDataPartTTLInfos::write(WriteBuffer & out) const if (it != moves_ttl.begin()) writeString(",", out); - writeString(R"({"expression":)", out); - writeString(doubleQuoteString(it->first), out); - writeString(R"(,"min":)", out); - writeIntText(it->second.min, out); - writeString(R"(,"max":)", out); - writeIntText(it->second.max, out); - writeString("}", out); + write_info_for_expression(it->first, it->second); } writeString("]", out); } @@ -149,13 +163,22 @@ void MergeTreeDataPartTTLInfos::write(WriteBuffer & out) const if (it != recompression_ttl.begin()) writeString(",", out); - writeString(R"({"expression":)", out); - writeString(doubleQuoteString(it->first), out); - writeString(R"(,"min":)", out); - writeIntText(it->second.min, out); - writeString(R"(,"max":)", out); - writeIntText(it->second.max, out); - writeString("}", out); + write_info_for_expression(it->first, it->second); + } + writeString("]", out); + } + if (!group_by_ttl.empty()) + { + if (!moves_ttl.empty() || !columns_ttl.empty() || !recompression_ttl.empty() || table_ttl.min) + writeString(",", out); + + writeString(R"("group_by":[)", out); + for (auto it = group_by_ttl.begin(); it != group_by_ttl.end(); ++it) + { + if (it != group_by_ttl.begin()) + writeString(",", out); + + write_info_for_expression(it->first, it->second); } writeString("]", out); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h index 17239e2618a..8ab6d6089db 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h @@ -49,10 +49,11 @@ struct MergeTreeDataPartTTLInfos TTLInfoMap recompression_ttl; + TTLInfoMap group_by_ttl; + /// Return the smallest max recompression TTL value time_t getMinimalMaxRecompressionTTL() const; - void read(ReadBuffer & in); void write(WriteBuffer & out) const; void update(const MergeTreeDataPartTTLInfos & other_infos); @@ -68,7 +69,7 @@ struct MergeTreeDataPartTTLInfos bool empty() const { - return !part_min_ttl && moves_ttl.empty() && recompression_ttl.empty(); + return !part_min_ttl && moves_ttl.empty() && recompression_ttl.empty() && group_by_ttl.empty(); } }; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index e1284fe8d92..68c409eb85c 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -376,6 +376,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa if (metadata_snapshot->hasRowsTTL()) updateTTL(metadata_snapshot->getRowsTTL(), new_data_part->ttl_infos, new_data_part->ttl_infos.table_ttl, block, true); + for (const auto & ttl_entry : metadata_snapshot->getGroupByTTLs()) + updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.group_by_ttl[ttl_entry.result_column], block, true); + for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs()) updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true); diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index a4500e2aa7b..463a7c3b382 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -125,7 +125,7 @@ TTLTableDescription StorageInMemoryMetadata::getTableTTLs() const bool StorageInMemoryMetadata::hasAnyTableTTL() const { - return hasAnyMoveTTL() || hasRowsTTL() || hasAnyRecompressionTTL(); + return hasAnyMoveTTL() || hasRowsTTL() || hasAnyRecompressionTTL() || hasAnyGroupByTTL(); } TTLColumnsDescription StorageInMemoryMetadata::getColumnTTLs() const @@ -168,6 +168,16 @@ bool StorageInMemoryMetadata::hasAnyRecompressionTTL() const return !table_ttl.recompression_ttl.empty(); } +TTLDescriptions StorageInMemoryMetadata::getGroupByTTLs() const +{ + return table_ttl.group_by_ttl; +} + +bool StorageInMemoryMetadata::hasAnyGroupByTTL() const +{ + return !table_ttl.group_by_ttl.empty(); +} + ColumnDependencies StorageInMemoryMetadata::getColumnDependencies(const NameSet & updated_columns) const { if (updated_columns.empty()) diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 3656edf71f4..cf9f38fe135 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -118,6 +118,10 @@ struct StorageInMemoryMetadata TTLDescriptions getRecompressionTTLs() const; bool hasAnyRecompressionTTL() const; + // Just wrapper for table TTLs, return info about recompression ttl + TTLDescriptions getGroupByTTLs() const; + bool hasAnyGroupByTTL() const; + /// Returns columns, which will be needed to calculate dependencies (skip /// indices, TTL expressions) if we update @updated_columns set of columns. ColumnDependencies getColumnDependencies(const NameSet & updated_columns) const; diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 7ae20ed024e..d890551893c 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -68,6 +68,10 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) {"recompression_ttl_info.expression", std::make_shared(std::make_shared())}, {"recompression_ttl_info.min", std::make_shared(std::make_shared())}, {"recompression_ttl_info.max", std::make_shared(std::make_shared())}, + + {"group_by_ttl_info.expression", std::make_shared(std::make_shared())}, + {"group_by_ttl_info.min", std::make_shared(std::make_shared())}, + {"group_by_ttl_info.max", std::make_shared(std::make_shared())} } ) { @@ -184,6 +188,7 @@ void StorageSystemParts::processNextStorage(MutableColumns & columns_, const Sto columns_[i++]->insert(queryToString(part->default_codec->getCodecDesc())); add_ttl_info_map(part->ttl_infos.recompression_ttl); + add_ttl_info_map(part->ttl_infos.group_by_ttl); } } diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index c2c5898c70c..d8731dd4ab3 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -259,6 +259,7 @@ TTLTableDescription::TTLTableDescription(const TTLTableDescription & other) , rows_ttl(other.rows_ttl) , move_ttl(other.move_ttl) , recompression_ttl(other.recompression_ttl) + , group_by_ttl(other.group_by_ttl) { } @@ -275,6 +276,7 @@ TTLTableDescription & TTLTableDescription::operator=(const TTLTableDescription & rows_ttl = other.rows_ttl; move_ttl = other.move_ttl; recompression_ttl = other.recompression_ttl; + group_by_ttl = other.group_by_ttl; return *this; } @@ -295,7 +297,7 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( for (const auto & ttl_element_ptr : definition_ast->children) { auto ttl = TTLDescription::getTTLFromAST(ttl_element_ptr, columns, context, primary_key); - if (ttl.mode == TTLMode::DELETE || ttl.mode == TTLMode::GROUP_BY) + if (ttl.mode == TTLMode::DELETE) { if (seen_delete_ttl) throw Exception("More than one DELETE TTL expression is not allowed", ErrorCodes::BAD_TTL_EXPRESSION); @@ -306,6 +308,10 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( { result.recompression_ttl.emplace_back(std::move(ttl)); } + else if (ttl.mode == TTLMode::GROUP_BY) + { + result.group_by_ttl.emplace_back(std::move(ttl)); + } else { result.move_ttl.emplace_back(std::move(ttl)); diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index 4b0d4370a70..1cc3a832447 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -107,6 +107,8 @@ struct TTLTableDescription TTLDescriptions recompression_ttl; + TTLDescriptions group_by_ttl; + TTLTableDescription() = default; TTLTableDescription(const TTLTableDescription & other); TTLTableDescription & operator=(const TTLTableDescription & other); From a8f1786d952482e0e4224537ad27e6cf8bd92ae2 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 29 Dec 2020 18:19:11 +0300 Subject: [PATCH 04/12] fix TTL with GROUP BY --- src/DataStreams/TTLColumnAlgorithm.cpp | 1 - src/Storages/TTLDescription.cpp | 9 ++++++--- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/src/DataStreams/TTLColumnAlgorithm.cpp b/src/DataStreams/TTLColumnAlgorithm.cpp index 4747a605e3b..afab3af62a7 100644 --- a/src/DataStreams/TTLColumnAlgorithm.cpp +++ b/src/DataStreams/TTLColumnAlgorithm.cpp @@ -26,7 +26,6 @@ void TTLColumnAlgorithm::execute(Block & block) if (!block) return; - /// If we read not all table columns. E.g. while mutation. if (!block.has(column_name)) return; diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index d8731dd4ab3..f0c936b10c2 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -211,9 +211,12 @@ TTLDescription TTLDescription::getTTLFromAST( const auto & primary_key_expressions = primary_key.expression_list_ast->children; for (size_t i = ttl_element->group_by_key.size(); i < primary_key_expressions.size(); ++i) { - ASTPtr expr = makeASTFunction("any", primary_key_expressions[i]->clone()); - aggregations.emplace_back(pk_columns[i], std::move(expr)); - aggregation_columns_set.insert(pk_columns[i]); + if (!aggregation_columns_set.count(pk_columns[i])) + { + ASTPtr expr = makeASTFunction("any", primary_key_expressions[i]->clone()); + aggregations.emplace_back(pk_columns[i], std::move(expr)); + aggregation_columns_set.insert(pk_columns[i]); + } } for (const auto & column : columns.getOrdinary()) From 5822ee1f01e124a19ab9ab03e0ba85fd79914982 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Jan 2021 02:07:21 +0300 Subject: [PATCH 05/12] allow multiple rows TTL with WHERE expression --- src/DataStreams/TTLBlockInputStream.cpp | 4 + src/DataStreams/TTLDeleteAlgorithm.cpp | 6 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 ++ .../MergeTree/MergeTreeDataPartTTLInfo.cpp | 87 +++++++++---------- .../MergeTree/MergeTreeDataPartTTLInfo.h | 2 + .../MergeTree/MergeTreeDataWriter.cpp | 3 + src/Storages/StorageInMemoryMetadata.cpp | 10 +++ src/Storages/StorageInMemoryMetadata.h | 3 + src/Storages/TTLDescription.cpp | 20 +++-- src/Storages/TTLDescription.h | 5 +- .../0_stateless/01622_multiple_ttls.reference | 9 ++ .../0_stateless/01622_multiple_ttls.sql | 20 +++++ 12 files changed, 120 insertions(+), 55 deletions(-) create mode 100644 tests/queries/0_stateless/01622_multiple_ttls.reference create mode 100644 tests/queries/0_stateless/01622_multiple_ttls.sql diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 8c680f6875b..5154949ae71 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -44,6 +44,10 @@ TTLBlockInputStream::TTLBlockInputStream( algorithms.emplace_back(std::move(algorithm)); } + for (const auto & where_ttl : metadata_snapshot_->getRowsWhereTTL()) + algorithms.emplace_back(std::make_unique( + where_ttl, old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_)); + for (const auto & group_by_ttl : metadata_snapshot_->getGroupByTTLs()) algorithms.emplace_back(std::make_unique( group_by_ttl, old_ttl_infos.group_by_ttl[group_by_ttl.result_column], current_time_, force_, header, storage_)); diff --git a/src/DataStreams/TTLDeleteAlgorithm.cpp b/src/DataStreams/TTLDeleteAlgorithm.cpp index 9ff4eb767df..7227b40dad2 100644 --- a/src/DataStreams/TTLDeleteAlgorithm.cpp +++ b/src/DataStreams/TTLDeleteAlgorithm.cpp @@ -51,7 +51,11 @@ void TTLDeleteAlgorithm::execute(Block & block) void TTLDeleteAlgorithm::finalize(const MutableDataPartPtr & data_part) const { - data_part->ttl_infos.table_ttl = new_ttl_info; + if (description.where_expression) + data_part->ttl_infos.rows_where_ttl[description.result_column] = new_ttl_info; + else + data_part->ttl_infos.table_ttl = new_ttl_info; + data_part->ttl_infos.updatePartMinMaxTTL(new_ttl_info.min, new_ttl_info.max); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index cdf66ec43f6..e78ff09cfc4 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1142,6 +1142,12 @@ bool IMergeTreeDataPart::checkAllTTLCalculated(const StorageMetadataPtr & metada return false; } + for (const auto & rows_where_desc : metadata_snapshot->getRowsWhereTTL()) + { + if (!ttl_infos.rows_where_ttl.count(rows_where_desc.result_column)) + return false; + } + return true; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp index 3a0bb283b63..138e38e3b78 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp @@ -17,6 +17,12 @@ void MergeTreeDataPartTTLInfos::update(const MergeTreeDataPartTTLInfos & other_i updatePartMinMaxTTL(ttl_info.min, ttl_info.max); } + for (const auto & [name, ttl_info] : other_infos.rows_where_ttl) + { + rows_where_ttl[name].update(ttl_info); + updatePartMinMaxTTL(ttl_info.min, ttl_info.max); + } + for (const auto & [name, ttl_info] : other_infos.group_by_ttl) { group_by_ttl[name].update(ttl_info); @@ -91,6 +97,11 @@ void MergeTreeDataPartTTLInfos::read(ReadBuffer & in) const JSON & group_by = json["group_by"]; fill_ttl_info_map(group_by, group_by_ttl); } + if (json.has("rows_where")) + { + const JSON & rows_where = json["rows_where"]; + fill_ttl_info_map(rows_where, rows_where_ttl); + } } @@ -127,61 +138,41 @@ void MergeTreeDataPartTTLInfos::write(WriteBuffer & out) const writeString("}", out); } - auto write_info_for_expression = [&out](const auto & name, const auto & info) + auto write_infos = [&out](const auto & infos, const auto & type, bool is_first) { - writeString(R"({"expression":)", out); - writeString(doubleQuoteString(name), out); - writeString(R"(,"min":)", out); - writeIntText(info.min, out); - writeString(R"(,"max":)", out); - writeIntText(info.max, out); - writeString("}", out); + if (!is_first) + writeString(",", out); + + writeString(type, out); + writeString(R"(:[)", out); + for (auto it = infos.begin(); it != infos.end(); ++it) + { + if (it != infos.begin()) + writeString(",", out); + + writeString(R"({"expression":)", out); + writeString(doubleQuoteString(it->first), out); + writeString(R"(,"min":)", out); + writeIntText(it->second.min, out); + writeString(R"(,"max":)", out); + writeIntText(it->second.max, out); + writeString("}", out); + } + writeString("]", out); }; - if (!moves_ttl.empty()) - { - if (!columns_ttl.empty() || table_ttl.min) - writeString(",", out); - writeString(R"("moves":[)", out); - for (auto it = moves_ttl.begin(); it != moves_ttl.end(); ++it) - { - if (it != moves_ttl.begin()) - writeString(",", out); + bool is_first = columns_ttl.empty() && !table_ttl.min; + write_infos(moves_ttl, "moves", is_first); - write_info_for_expression(it->first, it->second); - } - writeString("]", out); - } - if (!recompression_ttl.empty()) - { - if (!moves_ttl.empty() || !columns_ttl.empty() || table_ttl.min) - writeString(",", out); + is_first &= moves_ttl.empty(); + write_infos(recompression_ttl, "recompression", is_first); - writeString(R"("recompression":[)", out); - for (auto it = recompression_ttl.begin(); it != recompression_ttl.end(); ++it) - { - if (it != recompression_ttl.begin()) - writeString(",", out); + is_first &= recompression_ttl.empty(); + write_infos(group_by_ttl, "group_by", is_first); - write_info_for_expression(it->first, it->second); - } - writeString("]", out); - } - if (!group_by_ttl.empty()) - { - if (!moves_ttl.empty() || !columns_ttl.empty() || !recompression_ttl.empty() || table_ttl.min) - writeString(",", out); + is_first &= group_by_ttl.empty(); + write_infos(rows_where_ttl, "rows_where", is_first); - writeString(R"("group_by":[)", out); - for (auto it = group_by_ttl.begin(); it != group_by_ttl.end(); ++it) - { - if (it != group_by_ttl.begin()) - writeString(",", out); - - write_info_for_expression(it->first, it->second); - } - writeString("]", out); - } writeString("}", out); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h index 8ab6d6089db..8b972116384 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h @@ -45,6 +45,8 @@ struct MergeTreeDataPartTTLInfos time_t part_min_ttl = 0; time_t part_max_ttl = 0; + TTLInfoMap rows_where_ttl; + TTLInfoMap moves_ttl; TTLInfoMap recompression_ttl; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 68c409eb85c..7c733c660d6 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -379,6 +379,9 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa for (const auto & ttl_entry : metadata_snapshot->getGroupByTTLs()) updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.group_by_ttl[ttl_entry.result_column], block, true); + for (const auto & ttl_entry : metadata_snapshot->getRowsWhereTTL()) + updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.rows_where_ttl[ttl_entry.result_column], block, true); + for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs()) updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.columns_ttl[name], block, true); diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 463a7c3b382..36947706474 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -148,6 +148,16 @@ bool StorageInMemoryMetadata::hasRowsTTL() const return table_ttl.rows_ttl.expression != nullptr; } +TTLDescriptions StorageInMemoryMetadata::getRowsWhereTTL() const +{ + return table_ttl.rows_where_ttl; +} + +bool StorageInMemoryMetadata::hasRowsWhereTTL() const +{ + return !table_ttl.rows_where_ttl.empty(); +} + TTLDescriptions StorageInMemoryMetadata::getMoveTTLs() const { return table_ttl.move_ttl; diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index cf9f38fe135..4a00457f7eb 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -109,6 +109,9 @@ struct StorageInMemoryMetadata TTLDescription getRowsTTL() const; bool hasRowsTTL() const; + TTLDescriptions getRowsWhereTTL() const; + bool hasRowsWhereTTL() const; + /// Just wrapper for table TTLs, return moves (to disks or volumes) parts of /// table TTL. TTLDescriptions getMoveTTLs() const; diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index f0c936b10c2..6cef9e53097 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -260,6 +260,7 @@ TTLDescription TTLDescription::getTTLFromAST( TTLTableDescription::TTLTableDescription(const TTLTableDescription & other) : definition_ast(other.definition_ast ? other.definition_ast->clone() : nullptr) , rows_ttl(other.rows_ttl) + , rows_where_ttl(other.rows_where_ttl) , move_ttl(other.move_ttl) , recompression_ttl(other.recompression_ttl) , group_by_ttl(other.group_by_ttl) @@ -277,6 +278,7 @@ TTLTableDescription & TTLTableDescription::operator=(const TTLTableDescription & definition_ast.reset(); rows_ttl = other.rows_ttl; + rows_where_ttl = other.rows_where_ttl; move_ttl = other.move_ttl; recompression_ttl = other.recompression_ttl; group_by_ttl = other.group_by_ttl; @@ -296,16 +298,24 @@ TTLTableDescription TTLTableDescription::getTTLForTableFromAST( result.definition_ast = definition_ast->clone(); - bool seen_delete_ttl = false; + bool have_unconditional_delete_ttl = false; for (const auto & ttl_element_ptr : definition_ast->children) { auto ttl = TTLDescription::getTTLFromAST(ttl_element_ptr, columns, context, primary_key); if (ttl.mode == TTLMode::DELETE) { - if (seen_delete_ttl) - throw Exception("More than one DELETE TTL expression is not allowed", ErrorCodes::BAD_TTL_EXPRESSION); - result.rows_ttl = ttl; - seen_delete_ttl = true; + if (!ttl.where_expression) + { + if (have_unconditional_delete_ttl) + throw Exception("More than one DELETE TTL expression without WHERE expression is not allowed", ErrorCodes::BAD_TTL_EXPRESSION); + + have_unconditional_delete_ttl = true; + result.rows_ttl = ttl; + } + else + { + result.rows_where_ttl.emplace_back(std::move(ttl)); + } } else if (ttl.mode == TTLMode::RECOMPRESS) { diff --git a/src/Storages/TTLDescription.h b/src/Storages/TTLDescription.h index 1cc3a832447..a2340ad6bcd 100644 --- a/src/Storages/TTLDescription.h +++ b/src/Storages/TTLDescription.h @@ -99,9 +99,12 @@ struct TTLTableDescription /// ^~~~~~~~~~~~~~~definition~~~~~~~~~~~~~~~^ ASTPtr definition_ast; - /// Rows removing TTL + /// Unconditional main removing rows TTL. Can be only one for table. TTLDescription rows_ttl; + /// Conditional removing rows TTLs. + TTLDescriptions rows_where_ttl; + /// Moving data TTL (to other disks or volumes) TTLDescriptions move_ttl; diff --git a/tests/queries/0_stateless/01622_multiple_ttls.reference b/tests/queries/0_stateless/01622_multiple_ttls.reference new file mode 100644 index 00000000000..9b3ac02560c --- /dev/null +++ b/tests/queries/0_stateless/01622_multiple_ttls.reference @@ -0,0 +1,9 @@ +1970-10-10 2 +1970-10-10 5 +1970-10-10 8 +2000-10-10 1 +2000-10-10 2 +2000-10-10 4 +2000-10-10 5 +2000-10-10 7 +2000-10-10 8 diff --git a/tests/queries/0_stateless/01622_multiple_ttls.sql b/tests/queries/0_stateless/01622_multiple_ttls.sql new file mode 100644 index 00000000000..f86256150b5 --- /dev/null +++ b/tests/queries/0_stateless/01622_multiple_ttls.sql @@ -0,0 +1,20 @@ +DROP TABLE IF EXISTS ttl_where; + +CREATE TABLE ttl_where +( + `d` Date, + `i` UInt32 +) +ENGINE = MergeTree +ORDER BY tuple() +TTL d + toIntervalYear(10) DELETE WHERE i % 3 = 0, + d + toIntervalYear(40) DELETE WHERE i % 3 = 1; + +INSERT INTO ttl_where SELECT toDate('2000-10-10'), number FROM numbers(10); +INSERT INTO ttl_where SELECT toDate('1970-10-10'), number FROM numbers(10); + +OPTIMIZE TABLE ttl_where FINAL; + +SELECT * FROM ttl_where ORDER BY d, i; + +DROP TABLE ttl_where; From 61d6a323dddd0c049c10ee1602c5fe75adf49f5b Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Jan 2021 03:40:07 +0300 Subject: [PATCH 06/12] multiple TTL with GROUP BY --- src/Parsers/ASTTTLElement.cpp | 12 ++-- src/Parsers/ASTTTLElement.h | 2 +- src/Parsers/ExpressionElementParsers.cpp | 64 +++++++++++-------- src/Parsers/ExpressionElementParsers.h | 8 +++ src/Parsers/ParserAlterQuery.cpp | 29 --------- src/Parsers/ParserAlterQuery.h | 8 --- src/Storages/TTLDescription.cpp | 25 ++++++-- .../0_stateless/01622_multiple_ttls.reference | 13 ++++ .../0_stateless/01622_multiple_ttls.sql | 26 +++++++- 9 files changed, 113 insertions(+), 74 deletions(-) diff --git a/src/Parsers/ASTTTLElement.cpp b/src/Parsers/ASTTTLElement.cpp index 39283a3168e..2d22c1b4307 100644 --- a/src/Parsers/ASTTTLElement.cpp +++ b/src/Parsers/ASTTTLElement.cpp @@ -20,7 +20,7 @@ ASTPtr ASTTTLElement::clone() const for (auto & expr : clone->group_by_key) expr = expr->clone(); - for (auto & [name, expr] : clone->group_by_aggregations) + for (auto & expr : clone->group_by_assignments) expr = expr->clone(); return clone; @@ -46,15 +46,15 @@ void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & st settings.ostr << ", "; (*it)->formatImpl(settings, state, frame); } - if (!group_by_aggregations.empty()) + + if (!group_by_assignments.empty()) { settings.ostr << " SET "; - for (auto it = group_by_aggregations.begin(); it != group_by_aggregations.end(); ++it) + for (auto it = group_by_assignments.begin(); it != group_by_assignments.end(); ++it) { - if (it != group_by_aggregations.begin()) + if (it != group_by_assignments.begin()) settings.ostr << ", "; - settings.ostr << it->first << " = "; - it->second->formatImpl(settings, state, frame); + (*it)->formatImpl(settings, state, frame); } } } diff --git a/src/Parsers/ASTTTLElement.h b/src/Parsers/ASTTTLElement.h index aadd019b59c..ce011d76c7b 100644 --- a/src/Parsers/ASTTTLElement.h +++ b/src/Parsers/ASTTTLElement.h @@ -18,7 +18,7 @@ public: String destination_name; ASTs group_by_key; - std::vector> group_by_aggregations; + ASTs group_by_assignments; ASTPtr recompression_codec; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 726e28005e3..0bcbcac302a 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include @@ -1875,9 +1876,12 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserIdentifier parser_identifier; ParserStringLiteral parser_string_literal; ParserExpression parser_exp; - ParserExpressionList parser_expression_list(false); + ParserExpressionList parser_keys_list(false); ParserCodec parser_codec; + ParserList parser_assignment_list( + std::make_unique(), std::make_unique(TokenType::Comma)); + ASTPtr ttl_expr; if (!parser_exp.parse(pos, ttl_expr, expected)) return false; @@ -1911,9 +1915,9 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } ASTPtr where_expr; - ASTPtr ast_group_by_key; + ASTPtr group_by_key; ASTPtr recompression_codec; - std::vector> group_by_aggregations; + ASTPtr group_by_assignments; if (mode == TTLMode::MOVE) { @@ -1925,30 +1929,13 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } else if (mode == TTLMode::GROUP_BY) { - if (!parser_expression_list.parse(pos, ast_group_by_key, expected)) + if (!parser_keys_list.parse(pos, group_by_key, expected)) return false; if (s_set.ignore(pos)) { - while (true) - { - if (!group_by_aggregations.empty() && !s_comma.ignore(pos)) - break; - - ASTPtr name; - ASTPtr value; - if (!parser_identifier.parse(pos, name, expected)) - return false; - if (!s_eq.ignore(pos)) - return false; - if (!parser_exp.parse(pos, value, expected)) - return false; - - String name_str; - if (!tryGetIdentifierNameInto(name, name_str)) - return false; - group_by_aggregations.emplace_back(name_str, std::move(value)); - } + if (!parser_assignment_list.parse(pos, group_by_assignments, expected)) + return false; } } else if (mode == TTLMode::DELETE && s_where.ignore(pos)) @@ -1972,8 +1959,8 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (mode == TTLMode::GROUP_BY) { - ttl_element->group_by_key = std::move(ast_group_by_key->children); - ttl_element->group_by_aggregations = std::move(group_by_aggregations); + ttl_element->group_by_key = std::move(group_by_key->children); + ttl_element->group_by_assignments = std::move(group_by_assignments->children); } if (mode == TTLMode::RECOMPRESS) @@ -2008,4 +1995,31 @@ bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, ASTPtr & node, return false; } +bool ParserAssignment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + auto assignment = std::make_shared(); + node = assignment; + + ParserIdentifier p_identifier; + ParserToken s_equals(TokenType::Equals); + ParserExpression p_expression; + + ASTPtr column; + if (!p_identifier.parse(pos, column, expected)) + return false; + + if (!s_equals.ignore(pos, expected)) + return false; + + ASTPtr expression; + if (!p_expression.parse(pos, expression, expected)) + return false; + + tryGetIdentifierNameInto(column, assignment->column_name); + if (expression) + assignment->children.push_back(expression); + + return true; +} + } diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index 917f084a700..1eb17bfb0bd 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -468,4 +468,12 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +/// Part of the UPDATE command or TTL with GROUP BY of the form: col_name = expr +class ParserAssignment : public IParserBase +{ +protected: + const char * getName() const override{ return "column assignment"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; + } diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index f916537f438..5d20e27e486 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include @@ -651,34 +650,6 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe } -bool ParserAssignment::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) -{ - auto assignment = std::make_shared(); - node = assignment; - - ParserIdentifier p_identifier; - ParserToken s_equals(TokenType::Equals); - ParserExpression p_expression; - - ASTPtr column; - if (!p_identifier.parse(pos, column, expected)) - return false; - - if (!s_equals.ignore(pos, expected)) - return false; - - ASTPtr expression; - if (!p_expression.parse(pos, expression, expected)) - return false; - - tryGetIdentifierNameInto(column, assignment->column_name); - if (expression) - assignment->children.push_back(expression); - - return true; -} - - bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { auto query = std::make_shared(); diff --git a/src/Parsers/ParserAlterQuery.h b/src/Parsers/ParserAlterQuery.h index 514ef876430..b22b1c6ded2 100644 --- a/src/Parsers/ParserAlterQuery.h +++ b/src/Parsers/ParserAlterQuery.h @@ -63,12 +63,4 @@ public: }; -/// Part of the UPDATE command of the form: col_name = expr -class ParserAssignment : public IParserBase -{ -protected: - const char * getName() const override{ return "column assignment"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; -}; - } diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 6cef9e53097..06416bfbf36 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -7,12 +8,13 @@ #include #include #include +#include +#include #include #include #include - #include #include @@ -197,16 +199,31 @@ TTLDescription TTLDescription::getTTLFromAST( used_primary_key_columns_set.insert(pk_columns[i]); } - for (const auto & [name, _] : ttl_element->group_by_aggregations) + std::vector> aggregations; + for (const auto & ast : ttl_element->group_by_assignments) + { + const auto assignment = ast->as(); + auto expression = assignment.expression(); + + const auto * expression_func = expression->as(); + if (!expression_func || !AggregateFunctionFactory::instance().isAggregateFunctionName(expression_func->name)) + throw Exception(ErrorCodes::BAD_TTL_EXPRESSION, + "Invalid expression for assignment of column {}. Should be an aggregate function", assignment.column_name); + + auto type_literal = std::make_shared(columns.getPhysical(assignment.column_name).type->getName()); + expression = makeASTFunction("cast", expression->clone(), type_literal); + aggregations.emplace_back(assignment.column_name, std::move(expression)); + } + + for (const auto & [name, _] : aggregations) aggregation_columns_set.insert(name); - if (aggregation_columns_set.size() != ttl_element->group_by_aggregations.size()) + if (aggregation_columns_set.size() != ttl_element->group_by_assignments.size()) throw Exception( "Multiple aggregations set for one column in TTL Expression", ErrorCodes::BAD_TTL_EXPRESSION); result.group_by_keys = Names(pk_columns.begin(), pk_columns.begin() + ttl_element->group_by_key.size()); - auto aggregations = ttl_element->group_by_aggregations; const auto & primary_key_expressions = primary_key.expression_list_ast->children; for (size_t i = ttl_element->group_by_key.size(); i < primary_key_expressions.size(); ++i) diff --git a/tests/queries/0_stateless/01622_multiple_ttls.reference b/tests/queries/0_stateless/01622_multiple_ttls.reference index 9b3ac02560c..d9ebb694584 100644 --- a/tests/queries/0_stateless/01622_multiple_ttls.reference +++ b/tests/queries/0_stateless/01622_multiple_ttls.reference @@ -1,3 +1,4 @@ +TTL WHERE 1970-10-10 2 1970-10-10 5 1970-10-10 8 @@ -7,3 +8,15 @@ 2000-10-10 5 2000-10-10 7 2000-10-10 8 +TTL GROUP BY +1970-10-01 0 4950 +2000-10-01 0 450 +2000-10-01 1 460 +2000-10-01 2 470 +2000-10-01 3 480 +2000-10-01 4 490 +2000-10-01 5 500 +2000-10-01 6 510 +2000-10-01 7 520 +2000-10-01 8 530 +2000-10-01 9 540 diff --git a/tests/queries/0_stateless/01622_multiple_ttls.sql b/tests/queries/0_stateless/01622_multiple_ttls.sql index f86256150b5..aa2eeb5759b 100644 --- a/tests/queries/0_stateless/01622_multiple_ttls.sql +++ b/tests/queries/0_stateless/01622_multiple_ttls.sql @@ -1,3 +1,4 @@ +SELECT 'TTL WHERE'; DROP TABLE IF EXISTS ttl_where; CREATE TABLE ttl_where @@ -10,11 +11,34 @@ ORDER BY tuple() TTL d + toIntervalYear(10) DELETE WHERE i % 3 = 0, d + toIntervalYear(40) DELETE WHERE i % 3 = 1; +-- This test will fail at 2040-10-10 + INSERT INTO ttl_where SELECT toDate('2000-10-10'), number FROM numbers(10); INSERT INTO ttl_where SELECT toDate('1970-10-10'), number FROM numbers(10); - OPTIMIZE TABLE ttl_where FINAL; SELECT * FROM ttl_where ORDER BY d, i; DROP TABLE ttl_where; + +SELECT 'TTL GROUP BY'; +DROP TABLE IF EXISTS ttl_group_by; + +CREATE TABLE ttl_group_by +( + `d` Date, + `i` UInt32, + `v` UInt64 +) +ENGINE = MergeTree +ORDER BY (toStartOfMonth(d), i % 10) +TTL d + toIntervalYear(10) GROUP BY toStartOfMonth(d), i % 10 SET d = any(toStartOfMonth(d)), i = any(i % 10), v = sum(v), + d + toIntervalYear(40) GROUP BY toStartOfMonth(d) SET d = any(toStartOfMonth(d)), v = sum(v); + +INSERT INTO ttl_group_by SELECT toDate('2000-10-10'), number, number FROM numbers(100); +INSERT INTO ttl_group_by SELECT toDate('1970-10-10'), number, number FROM numbers(100); +OPTIMIZE TABLE ttl_group_by FINAL; + +SELECT * FROM ttl_group_by ORDER BY d, i; + +DROP TABLE ttl_group_by; From 58b9ef5a10a6d208b3ba68798015b87096ed42c3 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Jan 2021 17:04:03 +0300 Subject: [PATCH 07/12] fix TTL info serialization --- .../MergeTree/MergeTreeDataPartTTLInfo.cpp | 28 +++++++++++++------ 1 file changed, 19 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp index 138e38e3b78..d1916f31cc3 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp @@ -143,8 +143,8 @@ void MergeTreeDataPartTTLInfos::write(WriteBuffer & out) const if (!is_first) writeString(",", out); - writeString(type, out); - writeString(R"(:[)", out); + writeDoubleQuotedString(type, out); + writeString(":[", out); for (auto it = infos.begin(); it != infos.end(); ++it) { if (it != infos.begin()) @@ -162,16 +162,26 @@ void MergeTreeDataPartTTLInfos::write(WriteBuffer & out) const }; bool is_first = columns_ttl.empty() && !table_ttl.min; - write_infos(moves_ttl, "moves", is_first); + if (!moves_ttl.empty()) + { + write_infos(moves_ttl, "moves", is_first); + is_first = false; + } - is_first &= moves_ttl.empty(); - write_infos(recompression_ttl, "recompression", is_first); + if (!recompression_ttl.empty()) + { + write_infos(recompression_ttl, "recompression", is_first); + is_first = false; + } - is_first &= recompression_ttl.empty(); - write_infos(group_by_ttl, "group_by", is_first); + if (!group_by_ttl.empty()) + { + write_infos(group_by_ttl, "group_by", is_first); + is_first = false; + } - is_first &= group_by_ttl.empty(); - write_infos(rows_where_ttl, "rows_where", is_first); + if (!rows_where_ttl.empty()) + write_infos(rows_where_ttl, "rows_where", is_first); writeString("}", out); } From aed8c78d0d5ac77d7070bc39cda580ca6e92668f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Jan 2021 18:35:07 +0300 Subject: [PATCH 08/12] better check for existence of aggregate function --- src/Parsers/ExpressionElementParsers.cpp | 3 ++- src/Storages/TTLDescription.cpp | 25 ++++++++++++++++++++++-- 2 files changed, 25 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 0bcbcac302a..df67417d218 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1960,7 +1960,8 @@ bool ParserTTLElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (mode == TTLMode::GROUP_BY) { ttl_element->group_by_key = std::move(group_by_key->children); - ttl_element->group_by_assignments = std::move(group_by_assignments->children); + if (group_by_assignments) + ttl_element->group_by_assignments = std::move(group_by_assignments->children); } if (mode == TTLMode::RECOMPRESS) diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 06416bfbf36..42fdd76fc83 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -81,6 +82,24 @@ void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const Strin } } +class FindAggregateFunctionData +{ +public: + using TypeToVisit = ASTFunction; + bool has_aggregate_function = false; + + void visit(const ASTFunction & func, ASTPtr &) + { + /// Do not throw if found aggregate function inside another aggregate function, + /// because it will be checked, while creating expressions. + if (AggregateFunctionFactory::instance().isAggregateFunctionName(func.name)) + has_aggregate_function = true; + } +}; + +using FindAggregateFunctionFinderMatcher = OneTypeMatcher; +using FindAggregateFunctionVisitor = InDepthNodeVisitor; + } TTLDescription::TTLDescription(const TTLDescription & other) @@ -205,8 +224,10 @@ TTLDescription TTLDescription::getTTLFromAST( const auto assignment = ast->as(); auto expression = assignment.expression(); - const auto * expression_func = expression->as(); - if (!expression_func || !AggregateFunctionFactory::instance().isAggregateFunctionName(expression_func->name)) + FindAggregateFunctionVisitor::Data data{false}; + FindAggregateFunctionVisitor(data).visit(expression); + + if (!data.has_aggregate_function) throw Exception(ErrorCodes::BAD_TTL_EXPRESSION, "Invalid expression for assignment of column {}. Should be an aggregate function", assignment.column_name); From 60b88986bf5e0a30412e0b4cbcbd822914ca6a18 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 12 Jan 2021 19:42:49 +0300 Subject: [PATCH 09/12] minor changes near TTL computation --- src/DataStreams/ITTLAlgorithm.cpp | 8 ++++++-- src/DataStreams/ITTLAlgorithm.h | 7 ++++++- src/DataStreams/TTLAggregationAlgorithm.cpp | 4 ++-- src/DataStreams/TTLBlockInputStream.cpp | 7 +++++-- src/DataStreams/TTLColumnAlgorithm.cpp | 18 +++++++----------- src/DataStreams/TTLColumnAlgorithm.h | 4 +++- src/DataStreams/TTLDeleteAlgorithm.cpp | 4 ++-- src/DataStreams/TTLUpdateInfoAlgorithm.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 12 +++++------- src/Storages/TTLDescription.cpp | 4 ++-- 10 files changed, 39 insertions(+), 31 deletions(-) diff --git a/src/DataStreams/ITTLAlgorithm.cpp b/src/DataStreams/ITTLAlgorithm.cpp index f0e98e9ab1c..7513e0c6ce0 100644 --- a/src/DataStreams/ITTLAlgorithm.cpp +++ b/src/DataStreams/ITTLAlgorithm.cpp @@ -25,7 +25,8 @@ bool ITTLAlgorithm::isTTLExpired(time_t ttl) const return (ttl && (ttl <= current_time)); } -ColumnPtr ITTLAlgorithm::extractRequieredColumn(const ExpressionActionsPtr & expression, const Block & block, const String & result_column) +ColumnPtr ITTLAlgorithm::executeExpressionAndGetColumn( + const ExpressionActionsPtr & expression, const Block & block, const String & result_column) { if (!expression) return nullptr; @@ -37,7 +38,10 @@ ColumnPtr ITTLAlgorithm::extractRequieredColumn(const ExpressionActionsPtr & exp for (const auto & column_name : expression->getRequiredColumns()) block_copy.insert(block.getByName(column_name)); - expression->execute(block_copy); + /// Keep number of rows for const expression. + size_t num_rows = block.rows(); + expression->execute(block_copy, num_rows); + return block_copy.getByName(result_column).column; } diff --git a/src/DataStreams/ITTLAlgorithm.h b/src/DataStreams/ITTLAlgorithm.h index 28a371e9289..429ca4bcc61 100644 --- a/src/DataStreams/ITTLAlgorithm.h +++ b/src/DataStreams/ITTLAlgorithm.h @@ -23,10 +23,15 @@ public: bool isMinTTLExpired() const { return force || isTTLExpired(old_ttl_info.min); } bool isMaxTTLExpired() const { return isTTLExpired(old_ttl_info.max); } + /** This function is needed to avoid a conflict between already calculated columns and columns that needed to execute TTL. + * If result column is absent in block, all required columns are copied to new block and expression is executed on new block. + */ + static ColumnPtr executeExpressionAndGetColumn( + const ExpressionActionsPtr & expression, const Block & block, const String & result_column); + protected: bool isTTLExpired(time_t ttl) const; UInt32 getTimestampByIndex(const IColumn * column, size_t index) const; - static ColumnPtr extractRequieredColumn(const ExpressionActionsPtr & expression, const Block & block, const String & result_column); const TTLDescription description; const TTLInfo old_ttl_info; diff --git a/src/DataStreams/TTLAggregationAlgorithm.cpp b/src/DataStreams/TTLAggregationAlgorithm.cpp index 6cc1ac00b7e..ebe08159c55 100644 --- a/src/DataStreams/TTLAggregationAlgorithm.cpp +++ b/src/DataStreams/TTLAggregationAlgorithm.cpp @@ -56,8 +56,8 @@ void TTLAggregationAlgorithm::execute(Block & block) MutableColumns result_columns = header.cloneEmptyColumns(); MutableColumns aggregate_columns = header.cloneEmptyColumns(); - auto ttl_column = extractRequieredColumn(description.expression, block, description.result_column); - auto where_column = extractRequieredColumn(description.where_expression, block, description.where_result_column); + auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column); + auto where_column = executeExpressionAndGetColumn(description.where_expression, block, description.where_result_column); size_t rows_aggregated = 0; size_t current_key_start = 0; diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 5154949ae71..7dd5952bb07 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -60,19 +60,22 @@ TTLBlockInputStream::TTLBlockInputStream( for (const auto & [name, description] : metadata_snapshot_->getColumnTTLs()) { ExpressionActionsPtr default_expression; + String default_column_name; auto it = column_defaults.find(name); if (it != column_defaults.end()) { const auto & column = storage_columns.get(name); auto default_ast = it->second.expression->clone(); - default_ast = setAlias(addTypeConversionToAST(std::move(default_ast), column.type->getName()), it->first); + default_ast = addTypeConversionToAST(std::move(default_ast), column.type->getName()); auto syntax_result = TreeRewriter(storage_.global_context).analyze(default_ast, metadata_snapshot_->getColumns().getAllPhysical()); default_expression = ExpressionAnalyzer{default_ast, syntax_result, storage_.global_context}.getActions(true); + default_column_name = default_ast->getColumnName(); } algorithms.emplace_back(std::make_unique( - description, old_ttl_infos.columns_ttl[name], current_time_, force_, name, default_expression)); + description, old_ttl_infos.columns_ttl[name], current_time_, + force_, name, default_expression, default_column_name)); } } diff --git a/src/DataStreams/TTLColumnAlgorithm.cpp b/src/DataStreams/TTLColumnAlgorithm.cpp index afab3af62a7..140631ac0bf 100644 --- a/src/DataStreams/TTLColumnAlgorithm.cpp +++ b/src/DataStreams/TTLColumnAlgorithm.cpp @@ -9,10 +9,12 @@ TTLColumnAlgorithm::TTLColumnAlgorithm( time_t current_time_, bool force_, const String & column_name_, - const ExpressionActionsPtr & default_expression_) + const ExpressionActionsPtr & default_expression_, + const String & default_column_name_) : ITTLAlgorithm(description_, old_ttl_info_, current_time_, force_) , column_name(column_name_) , default_expression(default_expression_) + , default_column_name(default_column_name_) { if (!isMinTTLExpired()) { @@ -38,17 +40,11 @@ void TTLColumnAlgorithm::execute(Block & block) if (isMaxTTLExpired()) return; - //// TODO: use extractRequiredColumn - ColumnPtr default_column; - if (default_expression) - { - Block block_with_defaults; - block_with_defaults = block; - default_expression->execute(block_with_defaults); - default_column = block_with_defaults.getByName(column_name).column->convertToFullColumnIfConst(); - } + auto default_column = executeExpressionAndGetColumn(default_expression, block, default_column_name); + if (default_column) + default_column = default_column->convertToFullColumnIfConst(); - auto ttl_column = extractRequieredColumn(description.expression, block, description.result_column); + auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column); auto & column_with_type = block.getByName(column_name); const IColumn * values_column = column_with_type.column.get(); diff --git a/src/DataStreams/TTLColumnAlgorithm.h b/src/DataStreams/TTLColumnAlgorithm.h index b2824dba9b0..3b1c199292d 100644 --- a/src/DataStreams/TTLColumnAlgorithm.h +++ b/src/DataStreams/TTLColumnAlgorithm.h @@ -14,7 +14,8 @@ public: time_t current_time_, bool force_, const String & column_name_, - const ExpressionActionsPtr & default_expression_); + const ExpressionActionsPtr & default_expression_, + const String & default_column_name_); void execute(Block & block) override; void finalize(const MutableDataPartPtr & data_part) const override; @@ -22,6 +23,7 @@ public: private: const String column_name; const ExpressionActionsPtr default_expression; + const String default_column_name; bool is_fully_empty = true; }; diff --git a/src/DataStreams/TTLDeleteAlgorithm.cpp b/src/DataStreams/TTLDeleteAlgorithm.cpp index 7227b40dad2..c364bb06f3e 100644 --- a/src/DataStreams/TTLDeleteAlgorithm.cpp +++ b/src/DataStreams/TTLDeleteAlgorithm.cpp @@ -16,8 +16,8 @@ void TTLDeleteAlgorithm::execute(Block & block) if (!block || !isMinTTLExpired()) return; - auto ttl_column = extractRequieredColumn(description.expression, block, description.result_column); - auto where_column = extractRequieredColumn(description.where_expression, block, description.where_result_column); + auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column); + auto where_column = executeExpressionAndGetColumn(description.where_expression, block, description.where_result_column); MutableColumns result_columns; const auto & column_names = block.getNames(); diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp b/src/DataStreams/TTLUpdateInfoAlgorithm.cpp index ce4d4128eec..d5feb14658b 100644 --- a/src/DataStreams/TTLUpdateInfoAlgorithm.cpp +++ b/src/DataStreams/TTLUpdateInfoAlgorithm.cpp @@ -14,7 +14,7 @@ void TTLUpdateInfoAlgorithm::execute(Block & block) if (!block) return; - auto ttl_column = extractRequieredColumn(description.expression, block, description.result_column); + auto ttl_column = executeExpressionAndGetColumn(description.expression, block, description.result_column); for (size_t i = 0; i < block.rows(); ++i) { UInt32 cur_ttl = ITTLAlgorithm::getTimestampByIndex(ttl_column.get(), i); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 7c733c660d6..42fc24c8c8e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include @@ -95,23 +96,20 @@ void updateTTL( const Block & block, bool update_part_min_max_ttls) { - Block block_copy = block; - if (!block_copy.has(ttl_entry.result_column)) - ttl_entry.expression->execute(block_copy); + auto ttl_column = ITTLAlgorithm::executeExpressionAndGetColumn(ttl_entry.expression, block, ttl_entry.result_column); - const IColumn * column = block_copy.getByName(ttl_entry.result_column).column.get(); - if (const ColumnUInt16 * column_date = typeid_cast(column)) + if (const ColumnUInt16 * column_date = typeid_cast(ttl_column.get())) { const auto & date_lut = DateLUT::instance(); for (const auto & val : column_date->getData()) ttl_info.update(date_lut.fromDayNum(DayNum(val))); } - else if (const ColumnUInt32 * column_date_time = typeid_cast(column)) + else if (const ColumnUInt32 * column_date_time = typeid_cast(ttl_column.get())) { for (const auto & val : column_date_time->getData()) ttl_info.update(val); } - else if (const ColumnConst * column_const = typeid_cast(column)) + else if (const ColumnConst * column_const = typeid_cast(ttl_column.get())) { if (typeid_cast(&column_const->getDataColumn())) { diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 42fdd76fc83..19195e6ba6d 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -231,8 +232,7 @@ TTLDescription TTLDescription::getTTLFromAST( throw Exception(ErrorCodes::BAD_TTL_EXPRESSION, "Invalid expression for assignment of column {}. Should be an aggregate function", assignment.column_name); - auto type_literal = std::make_shared(columns.getPhysical(assignment.column_name).type->getName()); - expression = makeASTFunction("cast", expression->clone(), type_literal); + expression = addTypeConversionToAST(std::move(expression), columns.getPhysical(assignment.column_name).type->getName()); aggregations.emplace_back(assignment.column_name, std::move(expression)); } From 0e903552a06b1628a1dc9b2ca7e6b0383d856fba Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 13 Jan 2021 17:04:27 +0300 Subject: [PATCH 10/12] fix TTLs with WHERE --- src/DataStreams/TTLBlockInputStream.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 2 +- src/Storages/StorageInMemoryMetadata.cpp | 6 +++--- src/Storages/StorageInMemoryMetadata.h | 4 ++-- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/DataStreams/TTLBlockInputStream.cpp b/src/DataStreams/TTLBlockInputStream.cpp index 7dd5952bb07..4f141a03475 100644 --- a/src/DataStreams/TTLBlockInputStream.cpp +++ b/src/DataStreams/TTLBlockInputStream.cpp @@ -44,7 +44,7 @@ TTLBlockInputStream::TTLBlockInputStream( algorithms.emplace_back(std::move(algorithm)); } - for (const auto & where_ttl : metadata_snapshot_->getRowsWhereTTL()) + for (const auto & where_ttl : metadata_snapshot_->getRowsWhereTTLs()) algorithms.emplace_back(std::make_unique( where_ttl, old_ttl_infos.rows_where_ttl[where_ttl.result_column], current_time_, force_)); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 6ba351a4459..a937208b66a 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1142,7 +1142,7 @@ bool IMergeTreeDataPart::checkAllTTLCalculated(const StorageMetadataPtr & metada return false; } - for (const auto & rows_where_desc : metadata_snapshot->getRowsWhereTTL()) + for (const auto & rows_where_desc : metadata_snapshot->getRowsWhereTTLs()) { if (!ttl_infos.rows_where_ttl.count(rows_where_desc.result_column)) return false; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index b28f0979dc0..c3eafd2423e 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -377,7 +377,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa for (const auto & ttl_entry : metadata_snapshot->getGroupByTTLs()) updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.group_by_ttl[ttl_entry.result_column], block, true); - for (const auto & ttl_entry : metadata_snapshot->getRowsWhereTTL()) + for (const auto & ttl_entry : metadata_snapshot->getRowsWhereTTLs()) updateTTL(ttl_entry, new_data_part->ttl_infos, new_data_part->ttl_infos.rows_where_ttl[ttl_entry.result_column], block, true); for (const auto & [name, ttl_entry] : metadata_snapshot->getColumnTTLs()) diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 36947706474..f810c73c02a 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -125,7 +125,7 @@ TTLTableDescription StorageInMemoryMetadata::getTableTTLs() const bool StorageInMemoryMetadata::hasAnyTableTTL() const { - return hasAnyMoveTTL() || hasRowsTTL() || hasAnyRecompressionTTL() || hasAnyGroupByTTL(); + return hasAnyMoveTTL() || hasRowsTTL() || hasAnyRecompressionTTL() || hasAnyGroupByTTL() || hasAnyRowsWhereTTL(); } TTLColumnsDescription StorageInMemoryMetadata::getColumnTTLs() const @@ -148,12 +148,12 @@ bool StorageInMemoryMetadata::hasRowsTTL() const return table_ttl.rows_ttl.expression != nullptr; } -TTLDescriptions StorageInMemoryMetadata::getRowsWhereTTL() const +TTLDescriptions StorageInMemoryMetadata::getRowsWhereTTLs() const { return table_ttl.rows_where_ttl; } -bool StorageInMemoryMetadata::hasRowsWhereTTL() const +bool StorageInMemoryMetadata::hasAnyRowsWhereTTL() const { return !table_ttl.rows_where_ttl.empty(); } diff --git a/src/Storages/StorageInMemoryMetadata.h b/src/Storages/StorageInMemoryMetadata.h index 9a0f730f1f4..038416aff7d 100644 --- a/src/Storages/StorageInMemoryMetadata.h +++ b/src/Storages/StorageInMemoryMetadata.h @@ -109,8 +109,8 @@ struct StorageInMemoryMetadata TTLDescription getRowsTTL() const; bool hasRowsTTL() const; - TTLDescriptions getRowsWhereTTL() const; - bool hasRowsWhereTTL() const; + TTLDescriptions getRowsWhereTTLs() const; + bool hasAnyRowsWhereTTL() const; /// Just wrapper for table TTLs, return moves (to disks or volumes) parts of /// table TTL. From e5125b8c73797ce51d61c5f3bce487af3f3bdf37 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 28 Jan 2021 02:20:57 +0300 Subject: [PATCH 11/12] add comments and test for compatibility --- src/DataStreams/ITTLAlgorithm.h | 6 ++ src/DataStreams/TTLAggregationAlgorithm.h | 2 + src/DataStreams/TTLColumnAlgorithm.h | 2 + src/DataStreams/TTLDeleteAlgorithm.h | 2 + src/DataStreams/TTLUpdateInfoAlgorithm.h | 1 + .../MergeTree/MergeTreeDataPartTTLInfo.cpp | 13 ++-- .../MergeTree/MergeTreeDataPartTTLInfo.h | 3 +- src/Storages/System/StorageSystemParts.cpp | 7 +- src/Storages/TTLDescription.cpp | 11 ++- tests/integration/test_ttl_replicated/test.py | 75 +++++++++++++++++++ 10 files changed, 111 insertions(+), 11 deletions(-) diff --git a/src/DataStreams/ITTLAlgorithm.h b/src/DataStreams/ITTLAlgorithm.h index 429ca4bcc61..d87d43d8c0c 100644 --- a/src/DataStreams/ITTLAlgorithm.h +++ b/src/DataStreams/ITTLAlgorithm.h @@ -8,6 +8,10 @@ namespace DB { +/** + * Represents the actions, which are required to do + * with data, when TTL is expired: delete, aggregate, etc. + */ class ITTLAlgorithm { public: @@ -18,6 +22,8 @@ public: virtual ~ITTLAlgorithm() = default; virtual void execute(Block & block) = 0; + + /// Updates TTL metadata of the data_part. virtual void finalize(const MutableDataPartPtr & data_part) const = 0; bool isMinTTLExpired() const { return force || isTTLExpired(old_ttl_info.min); } diff --git a/src/DataStreams/TTLAggregationAlgorithm.h b/src/DataStreams/TTLAggregationAlgorithm.h index 977e755ca8b..c2f40bab6b9 100644 --- a/src/DataStreams/TTLAggregationAlgorithm.h +++ b/src/DataStreams/TTLAggregationAlgorithm.h @@ -7,6 +7,8 @@ namespace DB { +/// Aggregates rows according to 'TTL expr GROUP BY key' description. +/// Aggregation key must be the prefix of the sorting key. class TTLAggregationAlgorithm final : public ITTLAlgorithm { public: diff --git a/src/DataStreams/TTLColumnAlgorithm.h b/src/DataStreams/TTLColumnAlgorithm.h index 3b1c199292d..e09dd663af0 100644 --- a/src/DataStreams/TTLColumnAlgorithm.h +++ b/src/DataStreams/TTLColumnAlgorithm.h @@ -5,6 +5,8 @@ namespace DB { +/// Deletes (replaces to default) values in column according to column's TTL description. +/// If all values in column are replaced with defaults, this column won't be written to part. class TTLColumnAlgorithm final : public ITTLAlgorithm { public: diff --git a/src/DataStreams/TTLDeleteAlgorithm.h b/src/DataStreams/TTLDeleteAlgorithm.h index 36da59da46e..8ab3f8b63e8 100644 --- a/src/DataStreams/TTLDeleteAlgorithm.h +++ b/src/DataStreams/TTLDeleteAlgorithm.h @@ -5,6 +5,8 @@ namespace DB { +/// Deletes rows according to table TTL description with +/// possible optional condition in 'WHERE' clause. class TTLDeleteAlgorithm final : public ITTLAlgorithm { public: diff --git a/src/DataStreams/TTLUpdateInfoAlgorithm.h b/src/DataStreams/TTLUpdateInfoAlgorithm.h index 4a680c5bb3a..c1ef0e1c90d 100644 --- a/src/DataStreams/TTLUpdateInfoAlgorithm.h +++ b/src/DataStreams/TTLUpdateInfoAlgorithm.h @@ -5,6 +5,7 @@ namespace DB { +/// Calculates new ttl_info and does nothing with data. class TTLUpdateInfoAlgorithm : public ITTLAlgorithm { public: diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp index e9e4a2b2998..e130fbc1798 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.cpp @@ -70,7 +70,7 @@ void MergeTreeDataPartTTLInfos::read(ReadBuffer & in) updatePartMinMaxTTL(table_ttl.min, table_ttl.max); } - auto fill_ttl_info_map = [](const JSON & json_part, TTLInfoMap & ttl_info_map) + auto fill_ttl_info_map = [this](const JSON & json_part, TTLInfoMap & ttl_info_map, bool update_min_max) { for (auto elem : json_part) // NOLINT { @@ -79,28 +79,31 @@ void MergeTreeDataPartTTLInfos::read(ReadBuffer & in) ttl_info.max = elem["max"].getUInt(); String expression = elem["expression"].getString(); ttl_info_map.emplace(expression, ttl_info); + + if (update_min_max) + updatePartMinMaxTTL(ttl_info.min, ttl_info.max); } }; if (json.has("moves")) { const JSON & moves = json["moves"]; - fill_ttl_info_map(moves, moves_ttl); + fill_ttl_info_map(moves, moves_ttl, false); } if (json.has("recompression")) { const JSON & recompressions = json["recompression"]; - fill_ttl_info_map(recompressions, recompression_ttl); + fill_ttl_info_map(recompressions, recompression_ttl, false); } if (json.has("group_by")) { const JSON & group_by = json["group_by"]; - fill_ttl_info_map(group_by, group_by_ttl); + fill_ttl_info_map(group_by, group_by_ttl, true); } if (json.has("rows_where")) { const JSON & rows_where = json["rows_where"]; - fill_ttl_info_map(rows_where, rows_where_ttl); + fill_ttl_info_map(rows_where, rows_where_ttl, true); } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h index 8b972116384..9d1606ee44a 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h +++ b/src/Storages/MergeTree/MergeTreeDataPartTTLInfo.h @@ -71,7 +71,8 @@ struct MergeTreeDataPartTTLInfos bool empty() const { - return !part_min_ttl && moves_ttl.empty() && recompression_ttl.empty() && group_by_ttl.empty(); + /// part_min_ttl in minimum of rows, rows_where and group_by TTLs + return !part_min_ttl && moves_ttl.empty() && recompression_ttl.empty(); } }; diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 8845644f5db..bc5a96c6159 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -71,7 +71,11 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) {"group_by_ttl_info.expression", std::make_shared(std::make_shared())}, {"group_by_ttl_info.min", std::make_shared(std::make_shared())}, - {"group_by_ttl_info.max", std::make_shared(std::make_shared())} + {"group_by_ttl_info.max", std::make_shared(std::make_shared())}, + + {"rows_where_ttl_info.expression", std::make_shared(std::make_shared())}, + {"rows_where_ttl_info.min", std::make_shared(std::make_shared())}, + {"rows_where_ttl_info.max", std::make_shared(std::make_shared())} } ) { @@ -186,6 +190,7 @@ void StorageSystemParts::processNextStorage(MutableColumns & columns_, const Sto add_ttl_info_map(part->ttl_infos.recompression_ttl); add_ttl_info_map(part->ttl_infos.group_by_ttl); + add_ttl_info_map(part->ttl_infos.rows_where_ttl); /// _state column should be the latest. if (has_state_column) diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 19195e6ba6d..41c20b2714b 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -230,15 +230,13 @@ TTLDescription TTLDescription::getTTLFromAST( if (!data.has_aggregate_function) throw Exception(ErrorCodes::BAD_TTL_EXPRESSION, - "Invalid expression for assignment of column {}. Should be an aggregate function", assignment.column_name); + "Invalid expression for assignment of column {}. Should contain an aggregate function", assignment.column_name); expression = addTypeConversionToAST(std::move(expression), columns.getPhysical(assignment.column_name).type->getName()); aggregations.emplace_back(assignment.column_name, std::move(expression)); + aggregation_columns_set.insert(assignment.column_name); } - for (const auto & [name, _] : aggregations) - aggregation_columns_set.insert(name); - if (aggregation_columns_set.size() != ttl_element->group_by_assignments.size()) throw Exception( "Multiple aggregations set for one column in TTL Expression", @@ -247,6 +245,10 @@ TTLDescription TTLDescription::getTTLFromAST( result.group_by_keys = Names(pk_columns.begin(), pk_columns.begin() + ttl_element->group_by_key.size()); const auto & primary_key_expressions = primary_key.expression_list_ast->children; + + /// Wrap with 'any' aggregate function primary key columns, + /// which are not in 'GROUP BY' key and was not set explicitly. + /// The separate step, because not all primary key columns are ordinary columns. for (size_t i = ttl_element->group_by_key.size(); i < primary_key_expressions.size(); ++i) { if (!aggregation_columns_set.count(pk_columns[i])) @@ -257,6 +259,7 @@ TTLDescription TTLDescription::getTTLFromAST( } } + /// Wrap with 'any' aggregate function other columns, which was not set explicitly. for (const auto & column : columns.getOrdinary()) { if (!aggregation_columns_set.count(column.name) && !used_primary_key_columns_set.count(column.name)) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 9418aeaaf01..84b3340925c 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -9,6 +9,11 @@ cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', with_zookeeper=True) node2 = cluster.add_instance('node2', with_zookeeper=True) +node3 = cluster.add_instance('node3', with_zookeeper=True) +node4 = cluster.add_instance('node4', with_zookeeper=True, image='yandex/clickhouse-server', tag='20.12.4.5', stay_alive=True, with_installed_binary=True) + +node5 = cluster.add_instance('node5', with_zookeeper=True, image='yandex/clickhouse-server', tag='20.12.4.5', stay_alive=True, with_installed_binary=True) +node6 = cluster.add_instance('node6', with_zookeeper=True, image='yandex/clickhouse-server', tag='20.12.4.5', stay_alive=True, with_installed_binary=True) @pytest.fixture(scope="module") def started_cluster(): @@ -329,3 +334,73 @@ def test_ttl_empty_parts(started_cluster): error_msg = ' default.test_ttl_empty_parts (ReplicatedMergeTreeCleanupThread)' assert not node1.contains_in_log(error_msg) assert not node2.contains_in_log(error_msg) + +@pytest.mark.parametrize( + ('node_left', 'node_right', 'num_run'), + [(node1, node2, 0), (node3, node4, 1), (node5, node6, 2)] +) +def test_ttl_compatibility(started_cluster, node_left, node_right, num_run): + drop_table([node_left, node_right], "test_ttl") + drop_table([node_left, node_right], "test_ttl_group_by") + drop_table([node_left, node_right], "test_ttl_where") + + for node in [node_left, node_right]: + node.query( + ''' + CREATE TABLE test_ttl(date DateTime, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_{suff}', '{replica}') + ORDER BY id PARTITION BY toDayOfMonth(date) + TTL date + INTERVAL 3 SECOND + '''.format(suff=num_run, replica=node.name)) + + node.query( + ''' + CREATE TABLE test_ttl_group_by(date DateTime, id UInt32, val UInt64) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_group_by_{suff}', '{replica}') + ORDER BY id PARTITION BY toDayOfMonth(date) + TTL date + INTERVAL 3 SECOND GROUP BY id SET val = sum(val) + '''.format(suff=num_run, replica=node.name)) + + node.query( + ''' + CREATE TABLE test_ttl_where(date DateTime, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_where_{suff}', '{replica}') + ORDER BY id PARTITION BY toDayOfMonth(date) + TTL date + INTERVAL 3 SECOND DELETE WHERE id % 2 = 1 + '''.format(suff=num_run, replica=node.name)) + + node_left.query("INSERT INTO test_ttl VALUES (now(), 1)") + node_left.query("INSERT INTO test_ttl VALUES (toDateTime('2100-10-11 10:00:00'), 2)") + node_right.query("INSERT INTO test_ttl VALUES (now(), 3)") + node_right.query("INSERT INTO test_ttl VALUES (toDateTime('2100-10-11 10:00:00'), 4)") + + node_left.query("INSERT INTO test_ttl_group_by VALUES (now(), 0, 1)") + node_left.query("INSERT INTO test_ttl_group_by VALUES (now(), 0, 2)") + node_right.query("INSERT INTO test_ttl_group_by VALUES (now(), 0, 3)") + node_right.query("INSERT INTO test_ttl_group_by VALUES (now(), 0, 4)") + + node_left.query("INSERT INTO test_ttl_where VALUES (now(), 1)") + node_left.query("INSERT INTO test_ttl_where VALUES (now(), 2)") + node_right.query("INSERT INTO test_ttl_where VALUES (now(), 3)") + node_right.query("INSERT INTO test_ttl_where VALUES (now(), 4)") + + if node_left.with_installed_binary: + node_left.restart_with_latest_version() + + if node_right.with_installed_binary: + node_right.restart_with_latest_version() + + time.sleep(5) # Wait for TTL + + node_right.query("OPTIMIZE TABLE test_ttl FINAL") + node_right.query("OPTIMIZE TABLE test_ttl_group_by FINAL") + node_right.query("OPTIMIZE TABLE test_ttl_where FINAL") + + assert node_left.query("SELECT id FROM test_ttl ORDER BY id") == "2\n4\n" + assert node_right.query("SELECT id FROM test_ttl ORDER BY id") == "2\n4\n" + + assert node_left.query("SELECT val FROM test_ttl_group_by ORDER BY id") == "10\n" + assert node_right.query("SELECT val FROM test_ttl_group_by ORDER BY id") == "10\n" + + assert node_left.query("SELECT id FROM test_ttl_where ORDER BY id") == "2\n4\n" + assert node_right.query("SELECT id FROM test_ttl_where ORDER BY id") == "2\n4\n" From 7db0766192ced901b343224282f65156647f1443 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 28 Jan 2021 17:40:50 +0300 Subject: [PATCH 12/12] fix flaky checks --- tests/integration/test_ttl_replicated/test.py | 20 +++++++++---------- .../0_stateless/01280_ttl_where_group_by.sh | 2 +- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 84b3340925c..389e249790f 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -340,15 +340,15 @@ def test_ttl_empty_parts(started_cluster): [(node1, node2, 0), (node3, node4, 1), (node5, node6, 2)] ) def test_ttl_compatibility(started_cluster, node_left, node_right, num_run): - drop_table([node_left, node_right], "test_ttl") + drop_table([node_left, node_right], "test_ttl_delete") drop_table([node_left, node_right], "test_ttl_group_by") drop_table([node_left, node_right], "test_ttl_where") for node in [node_left, node_right]: node.query( ''' - CREATE TABLE test_ttl(date DateTime, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_{suff}', '{replica}') + CREATE TABLE test_ttl_delete(date DateTime, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/test_ttl_delete_{suff}', '{replica}') ORDER BY id PARTITION BY toDayOfMonth(date) TTL date + INTERVAL 3 SECOND '''.format(suff=num_run, replica=node.name)) @@ -369,10 +369,10 @@ def test_ttl_compatibility(started_cluster, node_left, node_right, num_run): TTL date + INTERVAL 3 SECOND DELETE WHERE id % 2 = 1 '''.format(suff=num_run, replica=node.name)) - node_left.query("INSERT INTO test_ttl VALUES (now(), 1)") - node_left.query("INSERT INTO test_ttl VALUES (toDateTime('2100-10-11 10:00:00'), 2)") - node_right.query("INSERT INTO test_ttl VALUES (now(), 3)") - node_right.query("INSERT INTO test_ttl VALUES (toDateTime('2100-10-11 10:00:00'), 4)") + node_left.query("INSERT INTO test_ttl_delete VALUES (now(), 1)") + node_left.query("INSERT INTO test_ttl_delete VALUES (toDateTime('2100-10-11 10:00:00'), 2)") + node_right.query("INSERT INTO test_ttl_delete VALUES (now(), 3)") + node_right.query("INSERT INTO test_ttl_delete VALUES (toDateTime('2100-10-11 10:00:00'), 4)") node_left.query("INSERT INTO test_ttl_group_by VALUES (now(), 0, 1)") node_left.query("INSERT INTO test_ttl_group_by VALUES (now(), 0, 2)") @@ -392,12 +392,12 @@ def test_ttl_compatibility(started_cluster, node_left, node_right, num_run): time.sleep(5) # Wait for TTL - node_right.query("OPTIMIZE TABLE test_ttl FINAL") + node_right.query("OPTIMIZE TABLE test_ttl_delete FINAL") node_right.query("OPTIMIZE TABLE test_ttl_group_by FINAL") node_right.query("OPTIMIZE TABLE test_ttl_where FINAL") - assert node_left.query("SELECT id FROM test_ttl ORDER BY id") == "2\n4\n" - assert node_right.query("SELECT id FROM test_ttl ORDER BY id") == "2\n4\n" + assert node_left.query("SELECT id FROM test_ttl_delete ORDER BY id") == "2\n4\n" + assert node_right.query("SELECT id FROM test_ttl_delete ORDER BY id") == "2\n4\n" assert node_left.query("SELECT val FROM test_ttl_group_by ORDER BY id") == "10\n" assert node_right.query("SELECT val FROM test_ttl_group_by ORDER BY id") == "10\n" diff --git a/tests/queries/0_stateless/01280_ttl_where_group_by.sh b/tests/queries/0_stateless/01280_ttl_where_group_by.sh index a83956b7c3d..9f30c7c5872 100755 --- a/tests/queries/0_stateless/01280_ttl_where_group_by.sh +++ b/tests/queries/0_stateless/01280_ttl_where_group_by.sh @@ -52,7 +52,7 @@ $CLICKHOUSE_CLIENT --query "drop table if exists ttl_01280_3" echo "ttl_01280_3" $CLICKHOUSE_CLIENT -n --query " -create table ttl_01280_3 (a Int, b Int, x Int64, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a set x = argMax(x, d), y = argMax(y, d), d = max(d); +create table ttl_01280_3 (a Int, b Int, x Int64, y Int, d DateTime) engine = MergeTree order by (a, b) ttl d + interval 1 second group by a set b = min(b), x = argMax(x, d), y = argMax(y, d), d = max(d); insert into ttl_01280_3 values (1, 1, 0, 4, now() + 10); insert into ttl_01280_3 values (1, 1, 10, 6, now() + 1); insert into ttl_01280_3 values (1, 2, 3, 7, now());