From 8ea4c2e26fbf51e8aa59f6ea3bf4e9b366182d67 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 24 Sep 2020 14:42:41 +0300 Subject: [PATCH 01/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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/94] 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 900580af026a10309ce3e3fe5789f4ea95468d7e Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 21 Jan 2021 20:33:08 +0300 Subject: [PATCH 11/94] Add parallel select when there is one part with level>0 in select final --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 81 ++++++++++++++----- .../optimized_select_final_one_part.xml | 20 +++++ ...t_merge_across_partitions_select_final.sql | 22 +++-- 3 files changed, 98 insertions(+), 25 deletions(-) create mode 100644 tests/performance/optimized_select_final_one_part.xml diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 4e1f307137a..9ac4b623f9f 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1305,6 +1305,12 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( data_settings->index_granularity, index_granularity_bytes); + const size_t min_marks_for_concurrent_read = roundRowsOrBytesToMarks( + settings.merge_tree_min_rows_for_concurrent_read, + settings.merge_tree_min_bytes_for_concurrent_read, + data_settings->index_granularity, + index_granularity_bytes); + if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; @@ -1347,25 +1353,60 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( { Pipes pipes; - for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) + /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition + /// with level > 0 then we won't postprocess this part and if num_streams > 1 we + /// can use parallel select on this part. + if (num_streams > 1 && settings.do_not_merge_across_partitions_select_final && + std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && + parts_to_merge_ranges[range_index]->data_part->info.level > 0) { - auto source_processor = std::make_shared( + MergeTreeReadPoolPtr pool = std::make_shared( + num_streams, + sum_marks, + min_marks_for_concurrent_read, + std::vector{*std::move(parts_to_merge_ranges[range_index])}, data, metadata_snapshot, - part_it->data_part, - max_block_size, - settings.preferred_block_size_bytes, - settings.preferred_max_column_in_block_size_bytes, - column_names, - part_it->ranges, - use_uncompressed_cache, query_info.prewhere_info, true, - reader_settings, - virt_columns, - part_it->part_index_in_query); + column_names, + MergeTreeReadPool::BackoffSettings(settings), + settings.preferred_block_size_bytes, + false); - pipes.emplace_back(std::move(source_processor)); + for (size_t i = 0; i < num_streams; ++i) + { + auto source = std::make_shared( + i, pool, min_marks_for_concurrent_read, max_block_size, + settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, + data, metadata_snapshot, use_uncompressed_cache, + query_info.prewhere_info, reader_settings, virt_columns); + + pipes.emplace_back(std::move(source)); + } + } + else + { + for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) + { + auto source_processor = std::make_shared( + data, + metadata_snapshot, + part_it->data_part, + max_block_size, + settings.preferred_block_size_bytes, + settings.preferred_max_column_in_block_size_bytes, + column_names, + part_it->ranges, + use_uncompressed_cache, + query_info.prewhere_info, + true, + reader_settings, + virt_columns, + part_it->part_index_in_query); + + pipes.emplace_back(std::move(source_processor)); + } } if (pipes.empty()) @@ -1380,6 +1421,13 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( plan = createPlanFromPipe(std::move(pipe), "with final"); } + auto expression_step = std::make_unique( + plan->getCurrentDataStream(), + metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); + + expression_step->setStepDescription("Calculate sorting key expression"); + plan->addStep(std::move(expression_step)); + /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition /// with level > 0 then we won't postprocess this part if (settings.do_not_merge_across_partitions_select_final && @@ -1390,13 +1438,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( continue; } - auto expression_step = std::make_unique( - plan->getCurrentDataStream(), - metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); - - expression_step->setStepDescription("Calculate sorting key expression"); - plan->addStep(std::move(expression_step)); - Names sort_columns = metadata_snapshot->getSortingKeyColumns(); SortDescription sort_description; size_t sort_columns_size = sort_columns.size(); diff --git a/tests/performance/optimized_select_final_one_part.xml b/tests/performance/optimized_select_final_one_part.xml new file mode 100644 index 00000000000..6ba8b0a71b0 --- /dev/null +++ b/tests/performance/optimized_select_final_one_part.xml @@ -0,0 +1,20 @@ + + + 1 + + + + CREATE TABLE optimized_select_final (t DateTime, x Int32, s String) + ENGINE = ReplacingMergeTree() + PARTITION BY toYYYYMM(t) ORDER BY x + + + INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), number, 'string' FROM numbers(500000000) + + OPTIMIZE TABLE optimized_select_final FINAL + + SELECT max(x) FROM optimized_select_final FINAL where s = 'string' FORMAT Null + + DROP TABLE IF EXISTS optimized_select_final + + diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql index d332946605d..c24990b598a 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql @@ -1,15 +1,27 @@ DROP TABLE IF EXISTS select_final; -CREATE TABLE select_final (t DateTime, x Int32) ENGINE = ReplacingMergeTree() PARTITION BY toYYYYMM(t) ORDER BY x; +CREATE TABLE select_final (t DateTime, x Int32, string String) ENGINE = ReplacingMergeTree() PARTITION BY toYYYYMM(t) ORDER BY (x, t); -INSERT INTO select_final SELECT toDate('2000-01-01'), number FROM numbers(2); -INSERT INTO select_final SELECT toDate('2000-01-01'), number + 1 FROM numbers(2); +INSERT INTO select_final SELECT toDate('2000-01-01'), number, '' FROM numbers(2); +INSERT INTO select_final SELECT toDate('2000-01-01'), number + 1, '' FROM numbers(2); -INSERT INTO select_final SELECT toDate('2020-01-01'), number FROM numbers(2); -INSERT INTO select_final SELECT toDate('2020-01-01'), number + 1 FROM numbers(2); +INSERT INTO select_final SELECT toDate('2020-01-01'), number, '' FROM numbers(2); +INSERT INTO select_final SELECT toDate('2020-01-01'), number + 1, '' FROM numbers(2); SELECT * FROM select_final FINAL ORDER BY x SETTINGS do_not_merge_across_partitions_select_final = 1; +TRUNCATE TABLE select_final; + +INSERT INTO select_final SELECT toDate('2000-01-01'), number, '' FROM numbers(2); +INSERT INTO select_final SELECT toDate('2000-01-01'), number, 'updated' FROM numbers(2); + +OPTIMIZE TABLE select_final FINAL; + +INSERT INTO select_final SELECT toDate('2020-01-01'), number, '' FROM numbers(2); +INSERT INTO select_final SELECT toDate('2020-01-01'), number, 'updated' FROM numbers(2); + +SELECT max(x) FROM select_final FINAL where string = 'updated' SETTINGS do_not_merge_across_partitions_select_final = 1; + DROP TABLE select_final; From 7706eb4f369617c6198ca1007fcd979ff3c73d39 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 21 Jan 2021 22:06:49 +0300 Subject: [PATCH 12/94] update test reference --- ...t_merge_across_partitions_select_final.reference | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference index 4c85a1d418a..facdf3dab26 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference @@ -1,6 +1,7 @@ -2000-01-01 00:00:00 0 -2020-01-01 00:00:00 0 -2000-01-01 00:00:00 1 -2020-01-01 00:00:00 1 -2000-01-01 00:00:00 2 -2020-01-01 00:00:00 2 +2000-01-01 00:00:00 0 +2020-01-01 00:00:00 0 +2000-01-01 00:00:00 1 +2020-01-01 00:00:00 1 +2000-01-01 00:00:00 2 +2020-01-01 00:00:00 2 +1 From 6d2b9ebbb27ac2e453f028c361f01459b046a196 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 22 Jan 2021 17:16:22 +0300 Subject: [PATCH 13/94] Update performance test --- tests/performance/optimized_select_final_one_part.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/optimized_select_final_one_part.xml b/tests/performance/optimized_select_final_one_part.xml index 6ba8b0a71b0..3724bc8f208 100644 --- a/tests/performance/optimized_select_final_one_part.xml +++ b/tests/performance/optimized_select_final_one_part.xml @@ -9,7 +9,7 @@ PARTITION BY toYYYYMM(t) ORDER BY x - INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), number, 'string' FROM numbers(500000000) + INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), number, 'string' FROM numbers(100000000) OPTIMIZE TABLE optimized_select_final FINAL From 39379bcd5c7478995abe1e990fedfd73b094c462 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sat, 23 Jan 2021 19:55:29 +0300 Subject: [PATCH 14/94] Update performance test --- tests/performance/optimized_select_final_one_part.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/performance/optimized_select_final_one_part.xml b/tests/performance/optimized_select_final_one_part.xml index 3724bc8f208..c7d505c89cb 100644 --- a/tests/performance/optimized_select_final_one_part.xml +++ b/tests/performance/optimized_select_final_one_part.xml @@ -13,7 +13,7 @@ OPTIMIZE TABLE optimized_select_final FINAL - SELECT max(x) FROM optimized_select_final FINAL where s = 'string' FORMAT Null + SELECT * FROM optimized_select_final FINAL where s = 'string' FORMAT Null DROP TABLE IF EXISTS optimized_select_final From 135426d3cbaa0e6a72547c9ea92d296658f3d774 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sat, 23 Jan 2021 21:16:59 +0300 Subject: [PATCH 15/94] Syntax updated, examples added. --- .../mergetree-family/mergetree.md | 54 +++++++++++++++++-- .../mergetree-family/mergetree.md | 54 +++++++++++++++++-- 2 files changed, 99 insertions(+), 9 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 084d05ec0a0..75fc42b6fc6 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -45,7 +45,10 @@ ORDER BY expr [PARTITION BY expr] [PRIMARY KEY expr] [SAMPLE BY expr] -[TTL expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'], ...] +[TTL expr + [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx' [, ...] ] + [WHERE conditions] + [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ] [SETTINGS name=value, ...] ``` @@ -455,7 +458,10 @@ ALTER TABLE example_table Table can have an expression for removal of expired rows, and multiple expressions for automatic move of parts between [disks or volumes](#table_engine-mergetree-multiple-volumes). When rows in the table expire, ClickHouse deletes all corresponding rows. For parts moving feature, all rows of a part must satisfy the movement expression criteria. ``` sql -TTL expr [DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'], ... +TTL expr + [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'][, DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'] ... + [WHERE conditions] + [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ``` Type of TTL rule may follow each TTL expression. It affects an action which is to be done once the expression is satisfied (reaches current time): @@ -464,9 +470,17 @@ Type of TTL rule may follow each TTL expression. It affects an action which is t - `TO DISK 'aaa'` - move part to the disk `aaa`; - `TO VOLUME 'bbb'` - move part to the disk `bbb`. -Examples: +With `WHERE` clause you may specify which of the expired rows to delete or move. -Creating a table with TTL +With `GROUP BY` clause you may [aggregate](../../../sql-reference/aggregate-functions/index.md) expired rows. `GROUP BY` key expression must be a prefix of the table primary key. + +If a column is part of primary key, but not present in `GROUP BY` key expression, in result rows it contains aggregated value across grouped rows. + +If a column is not present neither in primary key, nor in `SET` clause, in result row it contains any occasional value from grouped rows. + +**Examples** + +Creating a table with TTL: ``` sql CREATE TABLE example_table @@ -482,13 +496,43 @@ TTL d + INTERVAL 1 MONTH [DELETE], d + INTERVAL 2 WEEK TO DISK 'bbb'; ``` -Altering TTL of the table +Altering TTL of the table: ``` sql ALTER TABLE example_table MODIFY TTL d + INTERVAL 1 DAY; ``` +Creating a table, where the rows are expired after one month. The expired rows where dates are Mondays are deleted: + +``` sql +CREATE TABLE table_with_where +( + d DateTime, + a Int +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(d) +ORDER BY d +TTL d + INTERVAL 1 MONTH DELETE WHERE toDayOfWeek(d) = 1; +``` + +Creating a table, where expired rows are aggregated. In result rows `x` contains the maximum value accross the grouped rows, `y` — the minimum value, and `d` — any occasional value from grouped rows. + +``` sql +CREATE TABLE table_for_aggregation +( + d DateTime, + k1 Int, + k2 Int, + x Int, + y Int +) +ENGINE = MergeTree +ORDER BY k1, k2 +TTL d + INTERVAL 1 MONTH GROUP BY k1, k2 SET x = max(x), y = min(y); +``` + **Removing Data** Data with an expired TTL is removed when ClickHouse merges data parts. diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index 9b2a5eafca3..e21d4bc47e2 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -37,7 +37,10 @@ ORDER BY expr [PARTITION BY expr] [PRIMARY KEY expr] [SAMPLE BY expr] -[TTL expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'], ...] +[TTL expr + [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx' [, ...] ] + [WHERE conditions] + [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ] [SETTINGS name=value, ...] ``` @@ -443,7 +446,10 @@ ALTER TABLE example_table Для таблицы можно задать одно выражение для устаревания данных, а также несколько выражений, по срабатывании которых данные переместятся на [некоторый диск или том](#table_engine-mergetree-multiple-volumes). Когда некоторые данные в таблице устаревают, ClickHouse удаляет все соответствующие строки. ``` sql -TTL expr [DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'], ... +TTL expr + [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'][, DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'] ... + [WHERE conditions] + [GROUP BY key_expr [SET v1 = aggr_func(v1) [, v2 = aggr_func(v2) ...]] ] ``` За каждым TTL выражением может следовать тип действия, которое выполняется после достижения времени, соответствующего результату TTL выражения: @@ -452,7 +458,17 @@ TTL expr [DELETE|TO DISK 'aaa'|TO VOLUME 'bbb'], ... - `TO DISK 'aaa'` - переместить данные на диск `aaa`; - `TO VOLUME 'bbb'` - переместить данные на том `bbb`. -Примеры: +В секции `WHERE` можно задать условие удаления или перемещения устаревших строк. + +В секции `GROUP BY` можно [агрегировать](../../../sql-reference/aggregate-functions/index.md) данные из устаревших строк. Колонки, по которым агрегируются данные в `GROUP BY`, должны являться префиксом первичного ключа таблицы. + +Если колонка является частью первичного ключа, но не фигурирует в списке полей в `GROUP BY`, в результирующих строках она будет содержать агрегированные данные по сгруппированным строкам. + +Если колонка не является частью первичного ключа и не указана в секции `SET`, в результирующих строках она будет содержать случайное значение, взятое из одной из сгруппированных строк. + +**Примеры** + +Создание таблицы с TTL: ``` sql CREATE TABLE example_table @@ -468,13 +484,43 @@ TTL d + INTERVAL 1 MONTH [DELETE], d + INTERVAL 2 WEEK TO DISK 'bbb'; ``` -Изменение TTL +Изменение TTL: ``` sql ALTER TABLE example_table MODIFY TTL d + INTERVAL 1 DAY; ``` +Создание таблицы, в которой строки устаревают через месяц. Устаревшие строки удаляются, если дата выпадает на понедельник: + +``` sql +CREATE TABLE table_with_where +( + d DateTime, + a Int +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(d) +ORDER BY d +TTL d + INTERVAL 1 MONTH DELETE WHERE toDayOfWeek(d) = 1; +``` + +Создание таблицы, где устаревшие строки агрегируются. В результирующих строках колонка `x` содержит максимальное значение по сгруппированным строкам, `y` — минимальное значение, а `d` — случайное значение из одной из сгуппированных строк. + +``` sql +CREATE TABLE table_for_aggregation +( + d DateTime, + k1 Int, + k2 Int, + x Int, + y Int +) +ENGINE = MergeTree +ORDER BY k1, k2 +TTL d + INTERVAL 1 MONTH GROUP BY k1, k2 SET x = max(x), y = min(y); +``` + **Удаление данных** Данные с истекшим TTL удаляются, когда ClickHouse мёржит куски данных. From 66fe97d8bdaf271396a3bc9dfab493587c8a7183 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 25 Jan 2021 13:01:39 +0800 Subject: [PATCH 16/94] Per MergeTree table query limit --- src/Processors/Pipe.cpp | 2 + src/Processors/Pipe.h | 3 + src/Processors/QueryPlan/QueryIdHolder.cpp | 15 +++++ src/Processors/QueryPlan/QueryIdHolder.h | 21 +++++++ src/Storages/MergeTree/MergeTreeData.cpp | 21 +++++++ src/Storages/MergeTree/MergeTreeData.h | 10 +++ .../MergeTree/MergeTreeDataSelectExecutor.cpp | 48 ++++++++++---- .../MergeTree/MergeTreeDataSelectExecutor.h | 9 ++- src/Storages/MergeTree/MergeTreeSettings.h | 2 + ...01666_merge_tree_max_query_limit.reference | 14 +++++ .../01666_merge_tree_max_query_limit.sh | 63 +++++++++++++++++++ 11 files changed, 193 insertions(+), 15 deletions(-) create mode 100644 src/Processors/QueryPlan/QueryIdHolder.cpp create mode 100644 src/Processors/QueryPlan/QueryIdHolder.h create mode 100644 tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference create mode 100755 tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index e8943790e68..129bebf452a 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -105,6 +105,8 @@ Pipe::Holder & Pipe::Holder::operator=(Holder && rhs) for (auto & plan : rhs.query_plans) query_plans.emplace_back(std::move(plan)); + query_id_holder = std::move(rhs.query_id_holder); + return *this; } diff --git a/src/Processors/Pipe.h b/src/Processors/Pipe.h index 2d64de3e664..f21f4761977 100644 --- a/src/Processors/Pipe.h +++ b/src/Processors/Pipe.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include #include namespace DB @@ -108,6 +109,7 @@ public: /// This methods are from QueryPipeline. Needed to make conversion from pipeline to pipe possible. void addInterpreterContext(std::shared_ptr context) { holder.interpreter_context.emplace_back(std::move(context)); } void addStorageHolder(StoragePtr storage) { holder.storage_holders.emplace_back(std::move(storage)); } + void addQueryIdHolder(std::shared_ptr query_id_holder) { holder.query_id_holder = std::move(query_id_holder); } /// For queries with nested interpreters (i.e. StorageDistributed) void addQueryPlan(std::unique_ptr plan) { holder.query_plans.emplace_back(std::move(plan)); } @@ -128,6 +130,7 @@ private: std::vector storage_holders; std::vector table_locks; std::vector> query_plans; + std::shared_ptr query_id_holder; }; Holder holder; diff --git a/src/Processors/QueryPlan/QueryIdHolder.cpp b/src/Processors/QueryPlan/QueryIdHolder.cpp new file mode 100644 index 00000000000..87f6f892cd1 --- /dev/null +++ b/src/Processors/QueryPlan/QueryIdHolder.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ +QueryIdHolder::QueryIdHolder(const String & query_id_, const MergeTreeData & data_) : query_id(query_id_), data(data_) +{ +} + +QueryIdHolder::~QueryIdHolder() +{ + data.removeQueryId(query_id); +} + +} diff --git a/src/Processors/QueryPlan/QueryIdHolder.h b/src/Processors/QueryPlan/QueryIdHolder.h new file mode 100644 index 00000000000..ed8f9ec1d6b --- /dev/null +++ b/src/Processors/QueryPlan/QueryIdHolder.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ +class MergeTreeData; + +/// Holds the current query id and do something meaningful in destructor. +/// Currently it's used for cleaning query id in the MergeTreeData query set. +struct QueryIdHolder +{ + QueryIdHolder(const std::string & query_id_, const MergeTreeData & data_); + + ~QueryIdHolder(); + + std::string query_id; + const MergeTreeData & data; +}; + +} diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 276ac10aeaf..701e05430fb 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -114,6 +114,7 @@ namespace ErrorCodes extern const int NOT_ENOUGH_SPACE; extern const int ALTER_OF_COLUMN_IS_FORBIDDEN; extern const int SUPPORT_IS_DISABLED; + extern const int TOO_MANY_SIMULTANEOUS_QUERIES; } @@ -3988,4 +3989,24 @@ void MergeTreeData::setDataVolume(size_t bytes, size_t rows, size_t parts) total_active_size_rows.store(rows, std::memory_order_release); total_active_size_parts.store(parts, std::memory_order_release); } + +void MergeTreeData::insertQueryIdOrThrow(const String & query_id, size_t max_queries) const +{ + std::lock_guard lock(query_id_set_mutex); + if (query_id_set.find(query_id) != query_id_set.end()) + return; + if (query_id_set.size() >= max_queries) + throw Exception( + ErrorCodes::TOO_MANY_SIMULTANEOUS_QUERIES, "Too many simultaneous queries for table {}. Maximum is: {}", log_name, max_queries); + query_id_set.insert(query_id); +} + +void MergeTreeData::removeQueryId(const String & query_id) const +{ + std::lock_guard lock(query_id_set_mutex); + if (query_id_set.find(query_id) == query_id_set.end()) + LOG_WARNING(log, "We have query_id removed but it's not recorded. This is a bug"); + else + query_id_set.erase(query_id); +} } diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 9d021815888..425dcbfb316 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -702,6 +702,12 @@ public: /// section from config.xml. CompressionCodecPtr getCompressionCodecForPart(size_t part_size_compressed, const IMergeTreeDataPart::TTLInfos & ttl_infos, time_t current_time) const; + /// Record current query id where querying the table. Throw if there are already `max_queries` queries accessing the same table. + void insertQueryIdOrThrow(const String & query_id, size_t max_queries) const; + + /// Remove current query id after query finished. + void removeQueryId(const String & query_id) const; + /// Limiting parallel sends per one table, used in DataPartsExchange std::atomic_uint current_table_sends {0}; @@ -958,6 +964,10 @@ private: std::atomic total_active_size_bytes = 0; std::atomic total_active_size_rows = 0; std::atomic total_active_size_parts = 0; + + // Record all query ids which access the table. It's guarded by `query_id_set_mutex` and is always mutable. + mutable std::set query_id_set; + mutable std::mutex query_id_set_mutex; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 6b2e3c5a8a4..2dc88f08b30 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -33,6 +33,7 @@ #include #include #include +#include #include #include @@ -707,8 +708,9 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( if (parts_with_ranges.empty()) return std::make_unique(); + const auto data_settings = data.getSettings(); auto max_partitions_to_read - = settings.max_partitions_to_read.changed ? settings.max_partitions_to_read : data.getSettings()->max_partitions_to_read; + = settings.max_partitions_to_read.changed ? settings.max_partitions_to_read : data_settings->max_partitions_to_read; if (max_partitions_to_read > 0) { std::set partitions; @@ -722,6 +724,18 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( max_partitions_to_read); } + String query_id; + if (data_settings->max_concurrent_queries > 0) + { + if (data_settings->min_marks_to_honor_max_concurrent_queries > 0 + && sum_marks >= data_settings->min_marks_to_honor_max_concurrent_queries) + { + query_id = context.getCurrentQueryId(); + if (!query_id.empty()) + data.insertQueryIdOrThrow(query_id, data_settings->max_concurrent_queries); + } + } + ProfileEvents::increment(ProfileEvents::SelectedParts, parts_with_ranges.size()); ProfileEvents::increment(ProfileEvents::SelectedRanges, sum_ranges); ProfileEvents::increment(ProfileEvents::SelectedMarks, sum_marks); @@ -758,7 +772,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( virt_column_names, settings, reader_settings, - result_projection); + result_projection, + query_id); } else if ((settings.optimize_read_in_order || settings.optimize_aggregation_in_order) && query_info.input_order_info) { @@ -781,7 +796,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( virt_column_names, settings, reader_settings, - result_projection); + result_projection, + query_id); } else { @@ -795,7 +811,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( query_info, virt_column_names, settings, - reader_settings); + reader_settings, + query_id); } if (!plan) @@ -895,7 +912,7 @@ size_t minMarksForConcurrentRead( } -static QueryPlanPtr createPlanFromPipe(Pipe pipe, const std::string & description = "") +static QueryPlanPtr createPlanFromPipe(Pipe pipe, const String & query_id, const MergeTreeData & data, const std::string & description = "") { auto plan = std::make_unique(); @@ -903,6 +920,10 @@ static QueryPlanPtr createPlanFromPipe(Pipe pipe, const std::string & descriptio if (!description.empty()) storage_name += ' ' + description; + // Attach QueryIdHolder if needed + if (!query_id.empty()) + pipe.addQueryIdHolder(std::make_shared(query_id, data)); + auto step = std::make_unique(std::move(pipe), storage_name); plan->addStep(std::move(step)); return plan; @@ -918,7 +939,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( const SelectQueryInfo & query_info, const Names & virt_columns, const Settings & settings, - const MergeTreeReaderSettings & reader_settings) const + const MergeTreeReaderSettings & reader_settings, + const String & query_id) const { /// Count marks for each part. std::vector sum_marks_in_parts(parts.size()); @@ -1003,7 +1025,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( res.emplace_back(std::move(source)); } - return createPlanFromPipe(Pipe::unitePipes(std::move(res))); + return createPlanFromPipe(Pipe::unitePipes(std::move(res)), query_id, data); } else { @@ -1027,7 +1049,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( if (pipe.numOutputPorts() > 1) pipe.addTransform(std::make_shared(pipe.getHeader(), pipe.numOutputPorts())); - return createPlanFromPipe(std::move(pipe)); + return createPlanFromPipe(std::move(pipe), query_id, data); } } @@ -1051,7 +1073,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( const Names & virt_columns, const Settings & settings, const MergeTreeReaderSettings & reader_settings, - ActionsDAGPtr & out_projection) const + ActionsDAGPtr & out_projection, + const String & query_id) const { size_t sum_marks = 0; const InputOrderInfoPtr & input_order_info = query_info.input_order_info; @@ -1242,7 +1265,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( } } - auto plan = createPlanFromPipe(Pipe::unitePipes(std::move(pipes)), " with order"); + auto plan = createPlanFromPipe(Pipe::unitePipes(std::move(pipes)), query_id, data, " with order"); if (input_order_info->direction != 1) { @@ -1310,7 +1333,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( const Names & virt_columns, const Settings & settings, const MergeTreeReaderSettings & reader_settings, - ActionsDAGPtr & out_projection) const + ActionsDAGPtr & out_projection, + const String & query_id) const { const auto data_settings = data.getSettings(); size_t sum_marks = 0; @@ -1406,7 +1430,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( if (!out_projection) out_projection = createProjection(pipe.getHeader()); - plan = createPlanFromPipe(std::move(pipe), "with final"); + plan = createPlanFromPipe(std::move(pipe), query_id, data, "with final"); } /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index af4e3717749..c3b3020ebf5 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -58,7 +58,8 @@ private: const SelectQueryInfo & query_info, const Names & virt_columns, const Settings & settings, - const MergeTreeReaderSettings & reader_settings) const; + const MergeTreeReaderSettings & reader_settings, + const String & query_id) const; /// out_projection - save projection only with columns, requested to read QueryPlanPtr spreadMarkRangesAmongStreamsWithOrder( @@ -73,7 +74,8 @@ private: const Names & virt_columns, const Settings & settings, const MergeTreeReaderSettings & reader_settings, - ActionsDAGPtr & out_projection) const; + ActionsDAGPtr & out_projection, + const String & query_id) const; QueryPlanPtr spreadMarkRangesAmongStreamsFinal( RangesInDataParts && parts, @@ -86,7 +88,8 @@ private: const Names & virt_columns, const Settings & settings, const MergeTreeReaderSettings & reader_settings, - ActionsDAGPtr & out_projection) const; + ActionsDAGPtr & out_projection, + const String & query_id) const; /// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index. size_t getApproximateTotalRowsToRead( diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index 7f23a1a42ab..713bfffde05 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -111,6 +111,8 @@ struct Settings; M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm", 0) \ M(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \ M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \ + M(UInt64, max_concurrent_queries, 0, "Max number of concurrently executed queries related to the MergeTree table (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ + M(UInt64, min_marks_to_honor_max_concurrent_queries, 0, "Minimal number of marks to honor the MergeTree-level's max_concurrent_queries (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ \ /** Obsolete settings. Kept for backward compatibility only. */ \ M(UInt64, min_relative_delay_to_yield_leadership, 120, "Obsolete setting, does nothing.", 0) \ diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference new file mode 100644 index 00000000000..25880a7d740 --- /dev/null +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference @@ -0,0 +1,14 @@ +Spin up a long running query +Check if another query with some marks to read is throttled +yes +Check if another query with less marks to read is passed +0 100 +Modify min_marks_to_honor_max_concurrent_queries to 1 +Check if another query with less marks to read is throttled +yes +Modify max_concurrent_queries to 2 +Check if another query is passed +0 100 +Modify max_concurrent_queries back to 1 +Check if another query with less marks to read is throttled +yes diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh new file mode 100755 index 00000000000..0bf37673e91 --- /dev/null +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh @@ -0,0 +1,63 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --multiline --multiquery --query " +drop table if exists simple; + +create table simple (i int, j int) engine = MergeTree order by i +settings index_granularity = 1, max_concurrent_queries = 1, min_marks_to_honor_max_concurrent_queries = 2; + +insert into simple select number, number + 100 from numbers(10); +" + +echo "Spin up a long running query" +${CLICKHOUSE_CLIENT} --query "select sleepEachRow(1) from simple settings max_block_size = 1 format Null" --query_id "long_running_query" & +sleep 3 + +# query which reads marks >= min_marks_to_honor_max_concurrent_queries is throttled +echo "Check if another query with some marks to read is throttled" +${CLICKHOUSE_CLIENT} --query "select * from simple" 2> /dev/null; +CODE=$? +[ "$CODE" -ne "202" ] && echo "Expected error code: 202 but got: $CODE" && exit 1; +echo "yes" + +# query which reads marks less than min_marks_to_honor_max_concurrent_queries is allowed +echo "Check if another query with less marks to read is passed" +${CLICKHOUSE_CLIENT} --query "select * from simple where i = 0" + +# We can modify the settings to take effect for future queries +echo "Modify min_marks_to_honor_max_concurrent_queries to 1" +${CLICKHOUSE_CLIENT} --query "alter table simple modify setting min_marks_to_honor_max_concurrent_queries = 1" + +# Now smaller queries are also throttled +echo "Check if another query with less marks to read is throttled" +${CLICKHOUSE_CLIENT} --query "select * from simple where i = 0" 2> /dev/null; +CODE=$? +[ "$CODE" -ne "202" ] && echo "Expected error code: 202 but got: $CODE" && exit 1; +echo "yes" + +echo "Modify max_concurrent_queries to 2" +${CLICKHOUSE_CLIENT} --query "alter table simple modify setting max_concurrent_queries = 2" + +# Now more queries are accepted +echo "Check if another query is passed" +${CLICKHOUSE_CLIENT} --query "select * from simple where i = 0" + +echo "Modify max_concurrent_queries back to 1" +${CLICKHOUSE_CLIENT} --query "alter table simple modify setting max_concurrent_queries = 1" + +# Now queries are throttled again +echo "Check if another query with less marks to read is throttled" +${CLICKHOUSE_CLIENT} --query "select * from simple where i = 0" 2> /dev/null; +CODE=$? +[ "$CODE" -ne "202" ] && echo "Expected error code: 202 but got: $CODE" && exit 1; +echo "yes" + +wait + +${CLICKHOUSE_CLIENT} --multiline --multiquery --query " +drop table simple +" From 29a2ef3089c2ada0398341f7e080a0b0dd5b63ec Mon Sep 17 00:00:00 2001 From: kreuzerkrieg Date: Sat, 23 Jan 2021 17:20:15 +0200 Subject: [PATCH 17/94] Add IStoragePolicy interface --- src/Disks/IStoragePolicy.h | 62 +++++++++++++++++++ src/Disks/StoragePolicy.cpp | 24 +++++-- src/Disks/StoragePolicy.h | 43 +++++++------ src/Interpreters/Aggregator.cpp | 1 - src/Interpreters/Context.h | 4 +- src/Interpreters/SortedBlocksWriter.cpp | 2 +- .../Transforms/MergeSortingTransform.cpp | 2 +- src/Server/HTTPHandler.cpp | 2 +- src/Storages/IStorage.h | 4 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 1 - src/Storages/StorageDistributed.cpp | 1 - src/Storages/StorageMergeTree.cpp | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 1 - src/Storages/System/StorageSystemTables.cpp | 2 +- 14 files changed, 109 insertions(+), 41 deletions(-) create mode 100644 src/Disks/IStoragePolicy.h diff --git a/src/Disks/IStoragePolicy.h b/src/Disks/IStoragePolicy.h new file mode 100644 index 00000000000..a41ea87c328 --- /dev/null +++ b/src/Disks/IStoragePolicy.h @@ -0,0 +1,62 @@ +#pragma once +#include +#include +#include + +namespace DB +{ +class IStoragePolicy; +using StoragePolicyPtr = std::shared_ptr; +class IVolume; +using VolumePtr = std::shared_ptr; +using Volumes = std::vector; +class IDisk; +using DiskPtr = std::shared_ptr; +using Disks = std::vector; +class IReservation; +using ReservationPtr = std::unique_ptr; +using Reservations = std::vector; + +using String = std::string; + +class IStoragePolicy +{ +public: + virtual ~IStoragePolicy() = default; + virtual const String & getName() const = 0; + virtual const Volumes & getVolumes() const = 0; + /// Returns number [0., 1.] -- fraction of free space on disk + /// which should be kept with help of background moves + virtual double getMoveFactor() const = 0; + virtual bool isDefaultPolicy() const = 0; + /// Returns disks ordered by volumes priority + virtual Disks getDisks() const = 0; + /// Returns any disk + /// Used when it's not important, for example for + /// mutations files + virtual DiskPtr getAnyDisk() const = 0; + virtual DiskPtr getDiskByName(const String & disk_name) const = 0; + /// Get free space from most free disk + virtual UInt64 getMaxUnreservedFreeSpace() const = 0; + /// Reserves space on any volume with index > min_volume_index or returns nullptr + virtual ReservationPtr reserve(UInt64 bytes, size_t min_volume_index) const = 0; + /// Returns valid reservation or nullptr + virtual ReservationPtr reserve(UInt64 bytes) const = 0; + /// Reserves space on any volume or throws + virtual ReservationPtr reserveAndCheck(UInt64 bytes) const = 0; + /// Reserves 0 bytes on disk with max available space + /// Do not use this function when it is possible to predict size. + virtual ReservationPtr makeEmptyReservationOnLargestDisk() const = 0; + /// Get volume by index. + virtual VolumePtr getVolume(size_t index) const = 0; + virtual VolumePtr getVolumeByName(const String & volume_name) const = 0; + /// Checks if storage policy can be replaced by another one. + virtual void checkCompatibleWith(const StoragePolicyPtr & new_storage_policy) const = 0; + /// Find volume index, which contains disk + virtual size_t getVolumeIndexByDisk(const DiskPtr & disk_ptr) const = 0; + /// Check if we have any volume with stopped merges + virtual bool hasAnyVolumeWithDisabledMerges() const = 0; + virtual bool containsVolume(const String & volume_name) const = 0; +}; + +} diff --git a/src/Disks/StoragePolicy.cpp b/src/Disks/StoragePolicy.cpp index e3a937cae55..a1345879c83 100644 --- a/src/Disks/StoragePolicy.cpp +++ b/src/Disks/StoragePolicy.cpp @@ -93,17 +93,17 @@ StoragePolicy::StoragePolicy(String name_, Volumes volumes_, double move_factor_ } -StoragePolicy::StoragePolicy(const StoragePolicy & storage_policy, +StoragePolicy::StoragePolicy(StoragePolicyPtr storage_policy, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks) - : StoragePolicy(storage_policy.getName(), config, config_prefix, disks) + : StoragePolicy(storage_policy->getName(), config, config_prefix, disks) { for (auto & volume : volumes) { - if (storage_policy.volume_index_by_volume_name.count(volume->getName()) > 0) + if (storage_policy->containsVolume(volume->getName())) { - auto old_volume = storage_policy.getVolumeByName(volume->getName()); + auto old_volume = storage_policy->getVolumeByName(volume->getName()); try { auto new_volume = updateVolumeFromConfig(old_volume, config, config_prefix + ".volumes." + volume->getName(), disks); @@ -112,7 +112,7 @@ StoragePolicy::StoragePolicy(const StoragePolicy & storage_policy, catch (Exception & e) { /// Default policies are allowed to be missed in configuration. - if (e.code() != ErrorCodes::NO_ELEMENTS_IN_CONFIG || storage_policy.getName() != DEFAULT_STORAGE_POLICY_NAME) + if (e.code() != ErrorCodes::NO_ELEMENTS_IN_CONFIG || storage_policy->getName() != DEFAULT_STORAGE_POLICY_NAME) throw; Poco::Util::AbstractConfiguration::Keys keys; @@ -331,6 +331,11 @@ bool StoragePolicy::hasAnyVolumeWithDisabledMerges() const return false; } +bool StoragePolicy::containsVolume(const String & volume_name) const +{ + return volume_index_by_volume_name.contains(volume_name); +} + StoragePolicySelector::StoragePolicySelector( const Poco::Util::AbstractConfiguration & config, const String & config_prefix, @@ -345,6 +350,13 @@ StoragePolicySelector::StoragePolicySelector( throw Exception( "Storage policy name can contain only alphanumeric and '_' (" + backQuote(name) + ")", ErrorCodes::EXCESSIVE_ELEMENT_IN_CONFIG); + /* + * A customization point for StoragePolicy, here one can add his own policy, for example, based on policy's name + * if (name == "MyCustomPolicy") + * policies.emplace(name, std::make_shared(name, config, config_prefix + "." + name, disks)); + * else + */ + policies.emplace(name, std::make_shared(name, config, config_prefix + "." + name, disks)); LOG_INFO(&Poco::Logger::get("StoragePolicySelector"), "Storage policy {} loaded", backQuote(name)); } @@ -374,7 +386,7 @@ StoragePolicySelectorPtr StoragePolicySelector::updateFromConfig(const Poco::Uti /// Second pass, load. for (const auto & [name, policy] : policies) { - result->policies[name] = std::make_shared(*policy, config, config_prefix + "." + name, disks); + result->policies[name] = std::make_shared(policy, config, config_prefix + "." + name, disks); } return result; diff --git a/src/Disks/StoragePolicy.h b/src/Disks/StoragePolicy.h index 9135c27d1c0..6676ab19043 100644 --- a/src/Disks/StoragePolicy.h +++ b/src/Disks/StoragePolicy.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -23,14 +24,11 @@ namespace DB { -class StoragePolicy; -using StoragePolicyPtr = std::shared_ptr; - /** * Contains all information about volumes configuration for Storage. * Can determine appropriate Volume and Disk for each reservation. */ -class StoragePolicy +class StoragePolicy : public IStoragePolicy { public: StoragePolicy(String name_, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks); @@ -38,62 +36,63 @@ public: StoragePolicy(String name_, Volumes volumes_, double move_factor_); StoragePolicy( - const StoragePolicy & storage_policy, + StoragePolicyPtr storage_policy, const Poco::Util::AbstractConfiguration & config, const String & config_prefix, DiskSelectorPtr disks ); - bool isDefaultPolicy() const; + bool isDefaultPolicy() const override; /// Returns disks ordered by volumes priority - Disks getDisks() const; + Disks getDisks() const override; /// Returns any disk /// Used when it's not important, for example for /// mutations files - DiskPtr getAnyDisk() const; + DiskPtr getAnyDisk() const override; - DiskPtr getDiskByName(const String & disk_name) const; + DiskPtr getDiskByName(const String & disk_name) const override; /// Get free space from most free disk - UInt64 getMaxUnreservedFreeSpace() const; + UInt64 getMaxUnreservedFreeSpace() const override; - const String & getName() const { return name; } + const String & getName() const override{ return name; } /// Returns valid reservation or nullptr - ReservationPtr reserve(UInt64 bytes) const; + ReservationPtr reserve(UInt64 bytes) const override; /// Reserves space on any volume or throws - ReservationPtr reserveAndCheck(UInt64 bytes) const; + ReservationPtr reserveAndCheck(UInt64 bytes) const override; /// Reserves space on any volume with index > min_volume_index or returns nullptr - ReservationPtr reserve(UInt64 bytes, size_t min_volume_index) const; + ReservationPtr reserve(UInt64 bytes, size_t min_volume_index) const override; /// Find volume index, which contains disk - size_t getVolumeIndexByDisk(const DiskPtr & disk_ptr) const; + size_t getVolumeIndexByDisk(const DiskPtr & disk_ptr) const override; /// Reserves 0 bytes on disk with max available space /// Do not use this function when it is possible to predict size. - ReservationPtr makeEmptyReservationOnLargestDisk() const; + ReservationPtr makeEmptyReservationOnLargestDisk() const override; - const Volumes & getVolumes() const { return volumes; } + const Volumes & getVolumes() const override{ return volumes; } /// Returns number [0., 1.] -- fraction of free space on disk /// which should be kept with help of background moves - double getMoveFactor() const { return move_factor; } + double getMoveFactor() const override{ return move_factor; } /// Get volume by index. - VolumePtr getVolume(size_t index) const; + VolumePtr getVolume(size_t index) const override; - VolumePtr getVolumeByName(const String & volume_name) const; + VolumePtr getVolumeByName(const String & volume_name) const override; /// Checks if storage policy can be replaced by another one. - void checkCompatibleWith(const StoragePolicyPtr & new_storage_policy) const; + void checkCompatibleWith(const StoragePolicyPtr & new_storage_policy) const override; /// Check if we have any volume with stopped merges - bool hasAnyVolumeWithDisabledMerges() const; + bool hasAnyVolumeWithDisabledMerges() const override; + bool containsVolume(const String & volume_name) const override; private: Volumes volumes; const String name; diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index d83fef72882..8040091256c 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -19,7 +19,6 @@ #include #include #include -#include #include diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 8e15d0a4fed..e460fc732f9 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -102,8 +102,8 @@ using DiskPtr = std::shared_ptr; class DiskSelector; using DiskSelectorPtr = std::shared_ptr; using DisksMap = std::map; -class StoragePolicy; -using StoragePolicyPtr = std::shared_ptr; +class IStoragePolicy; +using StoragePolicyPtr = std::shared_ptr; using StoragePoliciesMap = std::map; class StoragePolicySelector; using StoragePolicySelectorPtr = std::shared_ptr; diff --git a/src/Interpreters/SortedBlocksWriter.cpp b/src/Interpreters/SortedBlocksWriter.cpp index 0dba09bc80f..f28bd53bd94 100644 --- a/src/Interpreters/SortedBlocksWriter.cpp +++ b/src/Interpreters/SortedBlocksWriter.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/src/Processors/Transforms/MergeSortingTransform.cpp b/src/Processors/Transforms/MergeSortingTransform.cpp index ce6d0ad1f6c..1806693db3a 100644 --- a/src/Processors/Transforms/MergeSortingTransform.cpp +++ b/src/Processors/Transforms/MergeSortingTransform.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include namespace ProfileEvents diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index 5006a817b5b..e161b5752ae 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 031b960fac1..1c0149ac261 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -50,8 +50,8 @@ class Pipe; class QueryPlan; using QueryPlanPtr = std::unique_ptr; -class StoragePolicy; -using StoragePolicyPtr = std::shared_ptr; +class IStoragePolicy; +using StoragePolicyPtr = std::shared_ptr; struct StreamLocalLimits; class EnabledQuota; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index f999aa67bbe..791c53633e9 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index afd7d6b876e..5227cd8a33e 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -1,7 +1,6 @@ #include #include -#include #include #include diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 070e6eb0483..83596b5b19d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -22,7 +22,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 70e90e9706a..2244b5c3ae1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -27,7 +27,6 @@ #include #include -#include #include diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 363a2a20828..132ed234323 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include #include #include #include From 578f36e4f3c84173e322c35470a5e1cc24dd0348 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 Jan 2021 16:58:49 +0300 Subject: [PATCH 18/94] Update test.Fix ya.make. --- src/Processors/ya.make | 1 + .../01666_merge_tree_max_query_limit.reference | 2 ++ .../0_stateless/01666_merge_tree_max_query_limit.sh | 12 +++++++++--- 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 2eb27be8899..d779cb320e6 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -117,6 +117,7 @@ SRCS( QueryPlan/MergingSortedStep.cpp QueryPlan/OffsetStep.cpp QueryPlan/PartialSortingStep.cpp + QueryPlan/QueryIdHolder.cpp QueryPlan/QueryPlan.cpp QueryPlan/ReadFromPreparedSource.cpp QueryPlan/ReadNothingStep.cpp diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference index 25880a7d740..9011a5d1204 100644 --- a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference @@ -12,3 +12,5 @@ Check if another query is passed Modify max_concurrent_queries back to 1 Check if another query with less marks to read is throttled yes +was cancelled +finished long_running_query default select sleepEachRow(0.01) from simple settings max_block_size = 1 format Null diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh index 0bf37673e91..27716aa8b28 100755 --- a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh @@ -4,18 +4,23 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +function wait_for_query_to_start() +{ + while [[ $($CLICKHOUSE_CURL -sS "$CLICKHOUSE_URL" -d "SELECT sum(read_rows) FROM system.processes WHERE query_id = '$1'") == 0 ]]; do sleep 0.1; done +} + ${CLICKHOUSE_CLIENT} --multiline --multiquery --query " drop table if exists simple; create table simple (i int, j int) engine = MergeTree order by i settings index_granularity = 1, max_concurrent_queries = 1, min_marks_to_honor_max_concurrent_queries = 2; -insert into simple select number, number + 100 from numbers(10); +insert into simple select number, number + 100 from numbers(1000); " echo "Spin up a long running query" -${CLICKHOUSE_CLIENT} --query "select sleepEachRow(1) from simple settings max_block_size = 1 format Null" --query_id "long_running_query" & -sleep 3 +${CLICKHOUSE_CLIENT} --query "select sleepEachRow(0.01) from simple settings max_block_size = 1 format Null" --query_id "long_running_query" 2>&1 | grep -o 'was cancelled' | head -1 & +wait_for_query_to_start 'long_running_query' # query which reads marks >= min_marks_to_honor_max_concurrent_queries is throttled echo "Check if another query with some marks to read is throttled" @@ -56,6 +61,7 @@ CODE=$? [ "$CODE" -ne "202" ] && echo "Expected error code: 202 but got: $CODE" && exit 1; echo "yes" +${CLICKHOUSE_CLIENT} --query "KILL QUERY WHERE query_id = 'long_running_query' SYNC" wait ${CLICKHOUSE_CLIENT} --multiline --multiquery --query " From 9d086f445da620ef59587ded0e142d979016c7aa Mon Sep 17 00:00:00 2001 From: guoleiyi Date: Wed, 27 Jan 2021 10:53:10 +0800 Subject: [PATCH 19/94] Should fail ddl query as soon as possible if table is shutdown --- src/Interpreters/DDLWorker.cpp | 7 +++++++ src/Storages/StorageReplicatedMergeTree.cpp | 1 + src/Storages/StorageReplicatedMergeTree.h | 1 + 3 files changed, 9 insertions(+) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index b1d9f872daa..e519b375c90 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -867,6 +867,13 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica( StorageReplicatedMergeTree::Status status; replicated_storage->getStatus(status); + // Should return as soon as possible if the table is shutdown by drop or other command. + if (status.is_partial_shutdown) { + LOG_WARNING(log, "Table is shutdown, task {} will not be executed.", task.entry_name); + task.execution_status = ExecutionStatus(ErrorCodes::UNFINISHED, "Cannot execute replicated DDL query, table is shutdown"); + return false; + } + /// Any replica which is leader tries to take lock if (status.is_leader && lock->tryLock()) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 70e90e9706a..f6e830f1570 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4791,6 +4791,7 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) res.can_become_leader = storage_settings_ptr->replicated_can_become_leader; res.is_readonly = is_readonly; res.is_session_expired = !zookeeper || zookeeper->expired(); + res.is_partial_shutdown = partial_shutdown_called; res.queue = queue.getStatus(); res.absolute_delay = getAbsoluteDelay(); /// NOTE: may be slightly inconsistent with queue status. diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index cf36cf82fc9..549f81c10e5 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -159,6 +159,7 @@ public: bool can_become_leader; bool is_readonly; bool is_session_expired; + bool is_partial_shutdown; ReplicatedMergeTreeQueue::Status queue; UInt32 parts_to_check; String zookeeper_path; From d707055ed609d6c2cbb63b2075b64cac4e4937c4 Mon Sep 17 00:00:00 2001 From: benbiti Date: Wed, 27 Jan 2021 13:56:20 +0800 Subject: [PATCH 20/94] update FINAL with max_final_threads setting to cn --- docs/zh/operations/settings/settings.md | 11 +++++++++++ docs/zh/sql-reference/statements/select/from.md | 6 ++++-- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/docs/zh/operations/settings/settings.md b/docs/zh/operations/settings/settings.md index f834ab74f5a..64625c19c6a 100644 --- a/docs/zh/operations/settings/settings.md +++ b/docs/zh/operations/settings/settings.md @@ -1310,3 +1310,14 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; **另请参阅** - [IN 运算符中的 NULL 处理](../../sql-reference/operators/in.md#in-null-processing) + +## max_final_threads {#max-final-threads} + +设置使用[FINAL](../../sql-reference/statements/select/from.md#select-from-final) 限定符的`SELECT`查询, 在数据读取阶段的最大并发线程数。 + +可能的值: + +- 正整数。 +- 0 or 1 — 禁用。 此时`SELECT` 查询单线程执行。 + +默认值: `16`。 diff --git a/docs/zh/sql-reference/statements/select/from.md b/docs/zh/sql-reference/statements/select/from.md index a8b49febab5..71b7cd319eb 100644 --- a/docs/zh/sql-reference/statements/select/from.md +++ b/docs/zh/sql-reference/statements/select/from.md @@ -25,11 +25,13 @@ toc_title: FROM - [Replicated](../../../engines/table-engines/mergetree-family/replication.md) 版本 `MergeTree` 引擎 - [View](../../../engines/table-engines/special/view.md), [Buffer](../../../engines/table-engines/special/buffer.md), [Distributed](../../../engines/table-engines/special/distributed.md),和 [MaterializedView](../../../engines/table-engines/special/materializedview.md) 在其他引擎上运行的引擎,只要是它们底层是 `MergeTree`-引擎表即可。 +现在使用 `FINAL` 修饰符 的 `SELECT` 查询启用了并发执行, 这会快一点。但是仍然存在缺陷 (见下)。 [max_final_threads](../../../operations/settings/settings.md#max-final-threads) 设置使用的最大线程数限制。 + ### 缺点 {#drawbacks} -使用的查询 `FINAL` 执行速度不如类似的查询那么快,因为: +使用的查询 `FINAL` 执行速度比类似的查询慢一点,因为: -- 查询在单个线程中执行,并在查询执行期间合并数据。 +- 在查询执行期间合并数据。 - 查询与 `FINAL` 除了读取查询中指定的列之外,还读取主键列。 **在大多数情况下,避免使用 `FINAL`.** 常见的方法是使用假设后台进程的不同查询 `MergeTree` 引擎还没有发生,并通过应用聚合(例如,丢弃重复项)来处理它。 {## TODO: examples ##} From 6693f77c322bbadcf7479350df258037376456d3 Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Wed, 27 Jan 2021 13:56:36 +0800 Subject: [PATCH 21/94] fix code style --- src/Interpreters/DDLWorker.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index e519b375c90..d3ebed228c7 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -868,7 +868,8 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica( replicated_storage->getStatus(status); // Should return as soon as possible if the table is shutdown by drop or other command. - if (status.is_partial_shutdown) { + if (status.is_partial_shutdown) + { LOG_WARNING(log, "Table is shutdown, task {} will not be executed.", task.entry_name); task.execution_status = ExecutionStatus(ErrorCodes::UNFINISHED, "Cannot execute replicated DDL query, table is shutdown"); return false; From 99a0401c82f651689ba47aad44410e9c1e9d7943 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 27 Jan 2021 14:36:55 +0300 Subject: [PATCH 22/94] Update 01666_merge_tree_max_query_limit.sh --- tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh index 27716aa8b28..e32a83c9560 100755 --- a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.sh @@ -19,7 +19,7 @@ insert into simple select number, number + 100 from numbers(1000); " echo "Spin up a long running query" -${CLICKHOUSE_CLIENT} --query "select sleepEachRow(0.01) from simple settings max_block_size = 1 format Null" --query_id "long_running_query" 2>&1 | grep -o 'was cancelled' | head -1 & +${CLICKHOUSE_CLIENT} --query "select sleepEachRow(0.01) from simple settings max_block_size = 1 format Null" --query_id "long_running_query" > /dev/null 2>&1 & wait_for_query_to_start 'long_running_query' # query which reads marks >= min_marks_to_honor_max_concurrent_queries is throttled From fc614d03c14cd4fab3c5ada845543ef1534cf5ee Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 27 Jan 2021 14:37:13 +0300 Subject: [PATCH 23/94] Update 01666_merge_tree_max_query_limit.reference --- .../0_stateless/01666_merge_tree_max_query_limit.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference index 9011a5d1204..a08a20dc95d 100644 --- a/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference +++ b/tests/queries/0_stateless/01666_merge_tree_max_query_limit.reference @@ -12,5 +12,4 @@ Check if another query is passed Modify max_concurrent_queries back to 1 Check if another query with less marks to read is throttled yes -was cancelled finished long_running_query default select sleepEachRow(0.01) from simple settings max_block_size = 1 format Null From e5125b8c73797ce51d61c5f3bce487af3f3bdf37 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 28 Jan 2021 02:20:57 +0300 Subject: [PATCH 24/94] 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 8ffc2bb72678fb1ab1a12bcbd7ee67824bbd353b Mon Sep 17 00:00:00 2001 From: benbiti Date: Thu, 28 Jan 2021 10:11:43 +0800 Subject: [PATCH 25/94] update aggregate-functions translations in Chinese Doc --- docs/zh/sql-reference/aggregate-functions/index.md | 14 +++++++------- .../sql-reference/aggregate-functions/reference.md | 4 ++-- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/zh/sql-reference/aggregate-functions/index.md b/docs/zh/sql-reference/aggregate-functions/index.md index 436a8f433ea..2344c3e6dc0 100644 --- a/docs/zh/sql-reference/aggregate-functions/index.md +++ b/docs/zh/sql-reference/aggregate-functions/index.md @@ -1,11 +1,12 @@ --- +toc_folder_title: 聚合函数 toc_priority: 33 -toc_title: 聚合函数 +toc_title: 简介 --- # 聚合函数 {#aggregate-functions} -聚合函数在 [正常](http://www.sql-tutorial.com/sql-aggregate-functions-sql-tutorial) 方式如预期的数据库专家。 +聚合函数如数据库专家预期的方式 [正常](http://www.sql-tutorial.com/sql-aggregate-functions-sql-tutorial) 工作。 ClickHouse还支持: @@ -14,7 +15,7 @@ ClickHouse还支持: ## 空处理 {#null-processing} -在聚合过程中,所有 `NULL`s被跳过。 +在聚合过程中,所有 `NULL` 被跳过。 **例:** @@ -30,7 +31,7 @@ ClickHouse还支持: └───┴──────┘ ``` -比方说,你需要在总的值 `y` 列: +比方说,你需要计算 `y` 列的总数: ``` sql SELECT sum(y) FROM t_null_big @@ -40,9 +41,8 @@ SELECT sum(y) FROM t_null_big │ 7 │ └────────┘ -该 `sum` 函数解释 `NULL` 作为 `0`. 特别是,这意味着,如果函数接收输入的选择,其中所有的值 `NULL`,那么结果将是 `0`,不 `NULL`. -现在你可以使用 `groupArray` 函数从创建一个数组 `y` 列: +现在你可以使用 `groupArray` 函数用 `y` 列创建一个数组: ``` sql SELECT groupArray(y) FROM t_null_big @@ -54,6 +54,6 @@ SELECT groupArray(y) FROM t_null_big └───────────────┘ ``` -`groupArray` 不包括 `NULL` 在生成的数组中。 +在 `groupArray` 生成的数组中不包括 `NULL`。 [原始文章](https://clickhouse.tech/docs/en/query_language/agg_functions/) diff --git a/docs/zh/sql-reference/aggregate-functions/reference.md b/docs/zh/sql-reference/aggregate-functions/reference.md index cf7dddb9b7e..3a224886a00 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference.md +++ b/docs/zh/sql-reference/aggregate-functions/reference.md @@ -1,9 +1,9 @@ --- toc_priority: 36 -toc_title: 聚合函数 +toc_title: 参考手册 --- -# 聚合函数引用 {#aggregate-functions-reference} +# 参考手册 {#aggregate-functions-reference} ## count {#agg_function-count} From b0d645eea87d01c21d0e1871345d42651cd728e0 Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Thu, 28 Jan 2021 20:12:00 +0800 Subject: [PATCH 26/94] check active replicas and detached tables --- src/Interpreters/DDLWorker.cpp | 13 ++++++++----- src/Storages/StorageReplicatedMergeTree.cpp | 1 - src/Storages/StorageReplicatedMergeTree.h | 1 - 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index d3ebed228c7..85445eb0cff 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -865,13 +865,16 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica( while (stopwatch.elapsedSeconds() <= MAX_EXECUTION_TIMEOUT_SEC) { StorageReplicatedMergeTree::Status status; - replicated_storage->getStatus(status); + // Has to get with zk fields to get active replicas field + replicated_storage->getStatus(status, true); - // Should return as soon as possible if the table is shutdown by drop or other command. - if (status.is_partial_shutdown) + // Should return as soon as possible if the table is dropped. + bool replica_dropped = replicated_storage->is_dropped; + bool all_replicas_likely_detached = status.active_replicas == 0 && !DatabaseCatalog::instance().isTableExist(replicated_storage->getStorageID(), context); + if (replica_dropped || all_replicas_likely_detached) { - LOG_WARNING(log, "Table is shutdown, task {} will not be executed.", task.entry_name); - task.execution_status = ExecutionStatus(ErrorCodes::UNFINISHED, "Cannot execute replicated DDL query, table is shutdown"); + LOG_WARNING(log, "Table is dropped or detached permantly, task {} will not be executed.", task.entry_name); + task.execution_status = ExecutionStatus(ErrorCodes::UNFINISHED, "Cannot execute replicated DDL query, table is dropped or detached permantly"); return false; } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f6e830f1570..70e90e9706a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4791,7 +4791,6 @@ void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) res.can_become_leader = storage_settings_ptr->replicated_can_become_leader; res.is_readonly = is_readonly; res.is_session_expired = !zookeeper || zookeeper->expired(); - res.is_partial_shutdown = partial_shutdown_called; res.queue = queue.getStatus(); res.absolute_delay = getAbsoluteDelay(); /// NOTE: may be slightly inconsistent with queue status. diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 549f81c10e5..cf36cf82fc9 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -159,7 +159,6 @@ public: bool can_become_leader; bool is_readonly; bool is_session_expired; - bool is_partial_shutdown; ReplicatedMergeTreeQueue::Status queue; UInt32 parts_to_check; String zookeeper_path; From 7db0766192ced901b343224282f65156647f1443 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 28 Jan 2021 17:40:50 +0300 Subject: [PATCH 27/94] 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()); From c6beaba12a1fab9b870e12d37e4917601da41b49 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Thu, 28 Jan 2021 22:18:49 +0300 Subject: [PATCH 28/94] Edit and translate into Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык, немного поправив английскую версию. --- .../sql-reference/statements/insert-into.md | 2 +- .../sql-reference/statements/select/index.md | 3 +- .../sql-reference/statements/insert-into.md | 2 +- .../sql-reference/statements/select/index.md | 109 +++++++++++++++++- 4 files changed, 110 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index 7acf4018812..61373b1c72f 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -13,7 +13,7 @@ Basic query format: INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... ``` -You can specify a list of columns to insert using the `(c1, c2, c3)`. You can also use an expression with column [matcher](../../sql-reference/statements/select/index.md#asterisk) such as `*` and/or [modifiers](../../sql-reference/statements/select/index.md#select-modifiers) such as [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#apply-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier). +You can specify a list of columns to insert using the `(c1, c2, c3)`. You can also use an expression with column [matcher](../../sql-reference/statements/select/index.md#asterisk) such as `*` and/or [modifiers](../../sql-reference/statements/select/index.md#select-modifiers) such as [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#except-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier). For example, consider the table: diff --git a/docs/en/sql-reference/statements/select/index.md b/docs/en/sql-reference/statements/select/index.md index 7c13772ffdf..e99ebef838c 100644 --- a/docs/en/sql-reference/statements/select/index.md +++ b/docs/en/sql-reference/statements/select/index.md @@ -278,5 +278,4 @@ Other ways to make settings see [here](../../../operations/settings/index.md). SELECT * FROM some_table SETTINGS optimize_read_in_order=1, cast_keep_nullable=1; ``` -[Original article](https://clickhouse.tech/docs/en/sql-reference/statements/select/) - +[Original article](https://clickhouse.tech/docs/en/sql-reference/statements/select/) diff --git a/docs/ru/sql-reference/statements/insert-into.md b/docs/ru/sql-reference/statements/insert-into.md index e3cea4aecc5..ba35f792e49 100644 --- a/docs/ru/sql-reference/statements/insert-into.md +++ b/docs/ru/sql-reference/statements/insert-into.md @@ -13,7 +13,7 @@ toc_title: INSERT INTO INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... ``` -Вы можете указать список столбцов для вставки, используя синтаксис `(c1, c2, c3)`. Также можно использовать выражение cо [звездочкой](../../sql-reference/statements/select/index.md#asterisk) и/или модификаторами, такими как `APPLY`, `EXCEPT`, `REPLACE`. +Вы можете указать список столбцов для вставки, используя синтаксис `(c1, c2, c3)`. Также можно использовать выражение cо [звездочкой](../../sql-reference/statements/select/index.md#asterisk) и/или модификаторами, такими как [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#except-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier). В качестве примера рассмотрим таблицу: diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index c37e82ae0be..536511ed5e8 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -162,6 +162,112 @@ Code: 42. DB::Exception: Received from localhost:9000. DB::Exception: Number of Подробнее смотрите в разделе «Настройки». Присутствует возможность использовать внешнюю сортировку (с сохранением временных данных на диск) и внешнюю агрегацию. +## Модификаторы запроса SELECT {#select-modifiers} + +Вы можете использовать следующие модификаторы в запросах `SELECT`. + +### APPLY {#apply-modifier} + +Вызывает некоторую функцию для каждой строки, которая возвращает внешнее табличное выражение запроса. + +**Синтаксис:** + +``` sql +SELECT APPLY( ) FROM [db.]table_name +``` + +**Пример:** + +``` sql +CREATE TABLE columns_transformers (i Int64, j Int16, k Int64) ENGINE = MergeTree ORDER by (i); +INSERT INTO columns_transformers VALUES (100, 10, 324), (120, 8, 23); +SELECT * APPLY(sum) FROM columns_transformers; +``` + +``` +┌─sum(i)─┬─sum(j)─┬─sum(k)─┐ +│ 220 │ 18 │ 347 │ +└────────┴────────┴────────┘ +``` + +### EXCEPT {#except-modifier} + +Указывает имена одного или нескольких столбцов для исключения из результата. + +**Синтаксис:** + +``` sql +SELECT EXCEPT ( col_name1 [, col_name2, col_name3, ...] ) FROM [db.]table_name +``` + +**Пример:** + +``` sql +SELECT * EXCEPT (i) from columns_transformers; +``` + +``` +┌──j─┬───k─┐ +│ 10 │ 324 │ +│ 8 │ 23 │ +└────┴─────┘ +``` + +### REPLACE {#replace-modifier} + +Указывает одно или несколько [выражений алиасов](../../../sql-reference/syntax.md#syntax-expression_aliases). Каждый алиас должен соответствовать имени столбца из запроса `SELECT *`. В списке столбцов из результата запроса имя столбца, соответствующее алиасу, заменяется выражением в модификаторе `REPLACE`. + +Этот модификатор не изменяет имена или порядок столбцов. Однако он может изменить значение и тип значения. + +**Синтаксис:** + +``` sql +SELECT REPLACE( AS col_name) from [db.]table_name +``` + +**Пример:** + +``` sql +SELECT * REPLACE(i + 1 AS i) from columns_transformers; +``` + +``` +┌───i─┬──j─┬───k─┐ +│ 101 │ 10 │ 324 │ +│ 121 │ 8 │ 23 │ +└─────┴────┴─────┘ +``` + +### Комбинации модификаторов {#modifier-combinations} + +Вы можете использовать каждый модификатор отдельно или комбинировать их. + +**Примеры:** + +Использование одного и того же модификатора несколько раз. + +``` sql +SELECT COLUMNS('[jk]') APPLY(toString) APPLY(length) APPLY(max) from columns_transformers; +``` + +``` +┌─max(length(toString(j)))─┬─max(length(toString(k)))─┐ +│ 2 │ 3 │ +└──────────────────────────┴──────────────────────────┘ +``` + +Использование нескольких модификаторов в одном запросе. + +``` sql +SELECT * REPLACE(i + 1 AS i) EXCEPT (j) APPLY(sum) from columns_transformers; +``` + +``` +┌─sum(plus(i, 1))─┬─sum(k)─┐ +│ 222 │ 347 │ +└─────────────────┴────────┘ +``` + ## SETTINGS в запросе SELECT {#settings-in-select} Вы можете задать значения необходимых настроек непосредственно в запросе `SELECT` в секции `SETTINGS`. Эти настройки действуют только в рамках данного запроса, а после его выполнения сбрасываются до предыдущего значения или значения по умолчанию. @@ -174,5 +280,4 @@ Code: 42. DB::Exception: Received from localhost:9000. DB::Exception: Number of SELECT * FROM some_table SETTINGS optimize_read_in_order=1, cast_keep_nullable=1; ``` -[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/select/) - +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/select/) From b57452446b36ae1aa024d5c9813716e4e4c33ad0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Jan 2021 22:42:57 +0300 Subject: [PATCH 29/94] client/suggest: add error code names --- programs/client/Suggest.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/client/Suggest.cpp b/programs/client/Suggest.cpp index 87083c2c27b..5e917422e7c 100644 --- a/programs/client/Suggest.cpp +++ b/programs/client/Suggest.cpp @@ -104,6 +104,8 @@ void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeo " UNION ALL " "SELECT cluster FROM system.clusters" " UNION ALL " + "SELECT name FROM system.errors" + " UNION ALL " "SELECT concat(func.name, comb.name) FROM system.functions AS func CROSS JOIN system.aggregate_function_combinators AS comb WHERE is_aggregate"; /// The user may disable loading of databases, tables, columns by setting suggestion_limit to zero. From a2bab85b789884f3bdce4863a9948df89846d000 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Jan 2021 23:05:41 +0300 Subject: [PATCH 30/94] client/suggest: set system_events_show_zero_values for query --- programs/client/Suggest.cpp | 11 ++++++++--- programs/client/Suggest.h | 2 +- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/programs/client/Suggest.cpp b/programs/client/Suggest.cpp index 5e917422e7c..e243cf6e6f1 100644 --- a/programs/client/Suggest.cpp +++ b/programs/client/Suggest.cpp @@ -1,5 +1,6 @@ #include "Suggest.h" +#include #include #include @@ -125,12 +126,16 @@ void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeo query << ") WHERE notEmpty(res)"; - fetch(connection, timeouts, query.str()); + Settings settings; + /// To show all rows from: + /// - system.errors + settings.system_events_show_zero_values = true; + fetch(connection, timeouts, query.str(), settings); } -void Suggest::fetch(Connection & connection, const ConnectionTimeouts & timeouts, const std::string & query) +void Suggest::fetch(Connection & connection, const ConnectionTimeouts & timeouts, const std::string & query, Settings & settings) { - connection.sendQuery(timeouts, query); + connection.sendQuery(timeouts, query, "" /* query_id */, QueryProcessingStage::Complete, &settings); while (true) { diff --git a/programs/client/Suggest.h b/programs/client/Suggest.h index 03332088cbe..0049bc08ebf 100644 --- a/programs/client/Suggest.h +++ b/programs/client/Suggest.h @@ -33,7 +33,7 @@ public: private: void loadImpl(Connection & connection, const ConnectionTimeouts & timeouts, size_t suggestion_limit); - void fetch(Connection & connection, const ConnectionTimeouts & timeouts, const std::string & query); + void fetch(Connection & connection, const ConnectionTimeouts & timeouts, const std::string & query, Settings & settings); void fillWordsFromBlock(const Block & block); /// Words are fetched asynchronously. From 1c68bf8545e8ecf801722edc51b6e94ce0cef9a7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Jan 2021 23:10:34 +0300 Subject: [PATCH 31/94] client/suggest: add other metrics --- programs/client/Suggest.cpp | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/programs/client/Suggest.cpp b/programs/client/Suggest.cpp index e243cf6e6f1..28cad80ffb6 100644 --- a/programs/client/Suggest.cpp +++ b/programs/client/Suggest.cpp @@ -107,6 +107,12 @@ void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeo " UNION ALL " "SELECT name FROM system.errors" " UNION ALL " + "SELECT event FROM system.events" + " UNION ALL " + "SELECT metric FROM system.asynchronous_metrics" + " UNION ALL " + "SELECT metric FROM system.metrics" + " UNION ALL " "SELECT concat(func.name, comb.name) FROM system.functions AS func CROSS JOIN system.aggregate_function_combinators AS comb WHERE is_aggregate"; /// The user may disable loading of databases, tables, columns by setting suggestion_limit to zero. @@ -129,6 +135,7 @@ void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeo Settings settings; /// To show all rows from: /// - system.errors + /// - system.events settings.system_events_show_zero_values = true; fetch(connection, timeouts, query.str(), settings); } From 212d5082b4feab509de7ac7b078b0bff532c5c0d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Jan 2021 23:15:47 +0300 Subject: [PATCH 32/94] client/suggest: add macros --- programs/client/Suggest.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/client/Suggest.cpp b/programs/client/Suggest.cpp index 28cad80ffb6..72f5a325b98 100644 --- a/programs/client/Suggest.cpp +++ b/programs/client/Suggest.cpp @@ -113,6 +113,8 @@ void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeo " UNION ALL " "SELECT metric FROM system.metrics" " UNION ALL " + "SELECT macro FROM system.macros" + " UNION ALL " "SELECT concat(func.name, comb.name) FROM system.functions AS func CROSS JOIN system.aggregate_function_combinators AS comb WHERE is_aggregate"; /// The user may disable loading of databases, tables, columns by setting suggestion_limit to zero. From 86f80105cb2a7a37e014da76c23902e93017015d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 25 Jan 2021 23:15:59 +0300 Subject: [PATCH 33/94] client/suggest: add policy --- programs/client/Suggest.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/programs/client/Suggest.cpp b/programs/client/Suggest.cpp index 72f5a325b98..a32e019e89b 100644 --- a/programs/client/Suggest.cpp +++ b/programs/client/Suggest.cpp @@ -115,6 +115,8 @@ void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeo " UNION ALL " "SELECT macro FROM system.macros" " UNION ALL " + "SELECT policy_name FROM system.storage_policies" + " UNION ALL " "SELECT concat(func.name, comb.name) FROM system.functions AS func CROSS JOIN system.aggregate_function_combinators AS comb WHERE is_aggregate"; /// The user may disable loading of databases, tables, columns by setting suggestion_limit to zero. From e7d8aa250cc9440e0b334aff68327b3eb2778be1 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Thu, 28 Jan 2021 23:57:04 +0300 Subject: [PATCH 34/94] Add links MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Добавил ссылки на модификаторы. --- docs/en/sql-reference/statements/insert-into.md | 2 +- docs/ru/sql-reference/statements/insert-into.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/insert-into.md b/docs/en/sql-reference/statements/insert-into.md index 2928e50224d..facc1b19dad 100644 --- a/docs/en/sql-reference/statements/insert-into.md +++ b/docs/en/sql-reference/statements/insert-into.md @@ -13,7 +13,7 @@ Basic query format: INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... ``` -You can specify a list of columns to insert using the `(c1, c2, c3)`. You can also use an expression with column [matcher](../../sql-reference/statements/select/index.md#asterisk) such as `*` and/or [modifiers](../../sql-reference/statements/select/index.md#select-modifiers) such as [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#apply-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier). +You can specify a list of columns to insert using the `(c1, c2, c3)`. You can also use an expression with column [matcher](../../sql-reference/statements/select/index.md#asterisk) such as `*` and/or [modifiers](../../sql-reference/statements/select/index.md#select-modifiers) such as [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#except-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier). For example, consider the table: diff --git a/docs/ru/sql-reference/statements/insert-into.md b/docs/ru/sql-reference/statements/insert-into.md index d83f6691f6b..61dc1170883 100644 --- a/docs/ru/sql-reference/statements/insert-into.md +++ b/docs/ru/sql-reference/statements/insert-into.md @@ -13,7 +13,7 @@ toc_title: INSERT INTO INSERT INTO [db.]table [(c1, c2, c3)] VALUES (v11, v12, v13), (v21, v22, v23), ... ``` -Вы можете указать список столбцов для вставки, используя синтаксис `(c1, c2, c3)`. Также можно использовать выражение cо [звездочкой](../../sql-reference/statements/select/index.md#asterisk) и/или модификаторами, такими как `APPLY`, `EXCEPT`, `REPLACE`. +Вы можете указать список столбцов для вставки, используя синтаксис `(c1, c2, c3)`. Также можно использовать выражение cо [звездочкой](../../sql-reference/statements/select/index.md#asterisk) и/или модификаторами, такими как [APPLY](../../sql-reference/statements/select/index.md#apply-modifier), [EXCEPT](../../sql-reference/statements/select/index.md#except-modifier), [REPLACE](../../sql-reference/statements/select/index.md#replace-modifier). В качестве примера рассмотрим таблицу: From 35f8a0810ac945403e40a4ff261f3c7304e96c1d Mon Sep 17 00:00:00 2001 From: taiyang-li Date: Fri, 29 Jan 2021 09:59:26 +0800 Subject: [PATCH 35/94] fix client bug while executing select --- src/Parsers/parseQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index 48a92534e74..650c0e40c8c 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -79,7 +79,7 @@ void writeQueryWithHighlightedErrorPositions( { const char * current_position_to_hilite = positions_to_hilite[position_to_hilite_idx].begin; - assert(current_position_to_hilite < end); + assert(current_position_to_hilite <= end); assert(current_position_to_hilite >= begin); out.write(pos, current_position_to_hilite - pos); From 585874bf46651890ee2c5f498dacfebe3b7d8c19 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 27 Jan 2021 22:44:22 +0300 Subject: [PATCH 36/94] tests: add a test for clickhouse-client autocompletion v2: Increase timeout for 01676_clickhouse_client_autocomplete https://github.com/ClickHouse/ClickHouse/pull/19584#discussion_r565727175 v3: Disable 01676_clickhouse_client_autocomplete in unbundled build (arcadia) autocomplete does not have to work fully unbundled build (since it lack of replxx). Similar to bd523a0aff1e1355300febcddadced05f393a15f v4: set expect timeout back to 1 and increase total timeout to 20 sec v4: set expect timeout back to 3 and increase total timeout to 22 (3*X+1) sec --- programs/client/Suggest.cpp | 5 + ...6_clickhouse_client_autocomplete.reference | 0 .../01676_clickhouse_client_autocomplete.sh | 114 ++++++++++++++++++ .../queries/0_stateless/arcadia_skip_list.txt | 1 + tests/queries/skip_list.json | 1 + 5 files changed, 121 insertions(+) create mode 100644 tests/queries/0_stateless/01676_clickhouse_client_autocomplete.reference create mode 100755 tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh diff --git a/programs/client/Suggest.cpp b/programs/client/Suggest.cpp index a32e019e89b..297b371d987 100644 --- a/programs/client/Suggest.cpp +++ b/programs/client/Suggest.cpp @@ -87,6 +87,11 @@ Suggest::Suggest() void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeouts, size_t suggestion_limit) { + /// + /// NOTE: Once you will update the completion list, + /// do not forget to update 01676_clickhouse_client_autocomplete.sh + /// + std::stringstream query; // STYLE_CHECK_ALLOW_STD_STRING_STREAM query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" "SELECT name FROM system.functions" diff --git a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.reference b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh new file mode 100755 index 00000000000..08e07044841 --- /dev/null +++ b/tests/queries/0_stateless/01676_clickhouse_client_autocomplete.sh @@ -0,0 +1,114 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +function test_completion_word() +{ + local w=$1 && shift + + local w_len=${#w} + local compword_begin=${w:0:$((w_len-3))} + local compword_end=${w:$((w_len-3))} + + # NOTE: here and below you should escape variables of the expect. + timeout 22s expect << EOF +log_user 0 +set timeout 3 +match_max 100000 +# A default timeout action is to do nothing, change it to fail +expect_after { + timeout { + exit 1 + } +} + +spawn bash -c "$CLICKHOUSE_CLIENT_BINARY $CLICKHOUSE_CLIENT_OPT" +expect ":) " + +# Make a query +send -- "SET $compword_begin" +expect "SET $compword_begin" + +# Wait for suggestions to load, they are loaded in background +set is_done 0 +while {\$is_done == 0} { + send -- "\\t" + expect { + "$compword_begin$compword_end" { + set is_done 1 + } + default { + sleep 1 + } + } +} + +send -- "\\3\\4" +expect eof +EOF +} + +# last 3 bytes will be completed, +# so take this in mind when you will update the list. +compwords_positive=( + # system.functions + concatAssumeInjective + # system.table_engines + ReplacingMergeTree + # system.formats + JSONEachRow + # system.table_functions + clusterAllReplicas + # system.data_type_families + SimpleAggregateFunction + # system.merge_tree_settings + write_ahead_log_interval_ms_to_fsync + # system.settings + max_concurrent_queries_for_all_users + # system.clusters + test_shard_localhost + # system.errors, also it is very rare to cover system_events_show_zero_values + CONDITIONAL_TREE_PARENT_NOT_FOUND + # system.events, also it is very rare to cover system_events_show_zero_values + WriteBufferFromFileDescriptorWriteFailed + # system.asynchronous_metrics, also this metric has zero value + # + # NOTE: that there is no ability to complete metrics like + # jemalloc.background_thread.num_runs, due to "." is used as a word breaker + # (and this cannot be changed -- db.table) + ReplicasMaxAbsoluteDelay + # system.metrics + PartsPreCommitted + # system.macros + default_path_test + # system.storage_policies, egh not uniq + default + # system.aggregate_function_combinators + uniqCombined64ForEach + + # FIXME: one may add separate case for suggestion_limit + # system.databases + system + # system.tables + aggregate_function_combinators + # system.columns + primary_key_bytes_in_memory_allocated + # system.dictionaries + # FIXME: none +) +for w in "${compwords_positive[@]}"; do + test_completion_word "$w" || echo "[FAIL] $w (positive)" +done + +# One negative is enough +compwords_negative=( + # system.clusters + test_shard_localhost_no_such_cluster +) +for w in "${compwords_negative[@]}"; do + test_completion_word "$w" && echo "[FAIL] $w (negative)" +done + +exit 0 diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 253eab720af..269d4ed72b1 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -198,3 +198,4 @@ 01659_test_base64Decode_mysql_compatibility 01675_data_type_coroutine 01671_aggregate_function_group_bitmap_data +01676_clickhouse_client_autocomplete diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 96da47ee9c2..727cb314748 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -90,6 +90,7 @@ "01300_client_save_history_when_terminated", "orc_output", "01370_client_autocomplete_word_break_characters", + "01676_clickhouse_client_autocomplete", "01193_metadata_loading", "01455_time_zones" ], From 71f4acd48b6a66aaf5ca1f7191ad5af3513aab66 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 29 Jan 2021 17:30:14 +0300 Subject: [PATCH 37/94] Use one pool for lonely parts, update tests --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 91 +++++++++++++------ .../optimized_select_final_one_part.xml | 2 +- ...e_across_partitions_select_final.reference | 1 + ...t_merge_across_partitions_select_final.sql | 1 + 4 files changed, 67 insertions(+), 28 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index e2aa33d2be3..cf8799052df 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1335,11 +1335,12 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( data_settings->index_granularity, index_granularity_bytes); - const size_t min_marks_for_concurrent_read = roundRowsOrBytesToMarks( + const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( settings.merge_tree_min_rows_for_concurrent_read, settings.merge_tree_min_bytes_for_concurrent_read, data_settings->index_granularity, - index_granularity_bytes); + index_granularity_bytes, + sum_marks); if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; @@ -1376,6 +1377,11 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( std::vector partition_plans; + /// If do_not_merge_across_partitions_select_final is true and num_streams > 1 + /// we will store lonely parts with level > 0 to use parallel select on them. + std::vector lonely_parts; + size_t total_rows_in_lonely_parts = 0; + for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index) { QueryPlanPtr plan; @@ -1385,35 +1391,15 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( /// If do_not_merge_across_partitions_select_final is true and there is only one part in partition /// with level > 0 then we won't postprocess this part and if num_streams > 1 we - /// can use parallel select on this part. + /// can use parallel select on such parts. We save such parts in one vector and then use + /// MergeTreeReadPool and MergeTreeThreadSelectBlockInputProcessor for parallel select. if (num_streams > 1 && settings.do_not_merge_across_partitions_select_final && std::distance(parts_to_merge_ranges[range_index], parts_to_merge_ranges[range_index + 1]) == 1 && parts_to_merge_ranges[range_index]->data_part->info.level > 0) { - MergeTreeReadPoolPtr pool = std::make_shared( - num_streams, - sum_marks, - min_marks_for_concurrent_read, - std::vector{*std::move(parts_to_merge_ranges[range_index])}, - data, - metadata_snapshot, - query_info.prewhere_info, - true, - column_names, - MergeTreeReadPool::BackoffSettings(settings), - settings.preferred_block_size_bytes, - false); - - for (size_t i = 0; i < num_streams; ++i) - { - auto source = std::make_shared( - i, pool, min_marks_for_concurrent_read, max_block_size, - settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, - data, metadata_snapshot, use_uncompressed_cache, - query_info.prewhere_info, reader_settings, virt_columns); - - pipes.emplace_back(std::move(source)); - } + total_rows_in_lonely_parts += parts_to_merge_ranges[range_index]->getRowsCount(); + lonely_parts.push_back(std::move(*parts_to_merge_ranges[range_index])); + continue; } else { @@ -1493,6 +1479,57 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( partition_plans.emplace_back(std::move(plan)); } + if (!lonely_parts.empty()) + { + Pipes pipes; + + size_t num_streams_for_lonely_parts = num_streams * lonely_parts.size(); + + MergeTreeReadPoolPtr pool = std::make_shared( + num_streams_for_lonely_parts, + sum_marks, + min_marks_for_concurrent_read, + std::move(lonely_parts), + data, + metadata_snapshot, + query_info.prewhere_info, + true, + column_names, + MergeTreeReadPool::BackoffSettings(settings), + settings.preferred_block_size_bytes, + false); + + LOG_TRACE(log, "Reading approx. {} rows with {} streams", total_rows_in_lonely_parts, num_streams_for_lonely_parts); + + for (size_t i = 0; i < num_streams_for_lonely_parts; ++i) + { + auto source = std::make_shared( + i, pool, min_marks_for_concurrent_read, max_block_size, + settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, + data, metadata_snapshot, use_uncompressed_cache, + query_info.prewhere_info, reader_settings, virt_columns); + + pipes.emplace_back(std::move(source)); + } + + auto pipe = Pipe::unitePipes(std::move(pipes)); + + /// Drop temporary columns, added by 'sorting_key_expr' + if (!out_projection) + out_projection = createProjection(pipe.getHeader()); + + QueryPlanPtr plan = createPlanFromPipe(std::move(pipe), "with final"); + + auto expression_step = std::make_unique( + plan->getCurrentDataStream(), + metadata_snapshot->getSortingKey().expression->getActionsDAG().clone()); + + expression_step->setStepDescription("Calculate sorting key expression"); + plan->addStep(std::move(expression_step)); + + partition_plans.emplace_back(std::move(plan)); + } + if (partition_plans.empty()) return {}; diff --git a/tests/performance/optimized_select_final_one_part.xml b/tests/performance/optimized_select_final_one_part.xml index c7d505c89cb..92c8eed859a 100644 --- a/tests/performance/optimized_select_final_one_part.xml +++ b/tests/performance/optimized_select_final_one_part.xml @@ -9,7 +9,7 @@ PARTITION BY toYYYYMM(t) ORDER BY x - INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), number, 'string' FROM numbers(100000000) + INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), number, 'string' FROM numbers(50000000) OPTIMIZE TABLE optimized_select_final FINAL diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference index facdf3dab26..963359440c6 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference @@ -5,3 +5,4 @@ 2000-01-01 00:00:00 2 2020-01-01 00:00:00 2 1 +16 diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql index c24990b598a..0e0f0325c7b 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql @@ -22,6 +22,7 @@ INSERT INTO select_final SELECT toDate('2020-01-01'), number, '' FROM numbers(2) INSERT INTO select_final SELECT toDate('2020-01-01'), number, 'updated' FROM numbers(2); SELECT max(x) FROM select_final FINAL where string = 'updated' SETTINGS do_not_merge_across_partitions_select_final = 1; +SELECT arrayUniq(thread_ids) FROM system.query_log ORDER BY event_time LIMIT 1; DROP TABLE select_final; From 78371e15dcc9b86f2fdf4b30637393dd22863be8 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 29 Jan 2021 21:00:08 +0300 Subject: [PATCH 38/94] Update test, reduce num_threads_for_lonely_parts if data is small --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 23 ++++++++++++------- ...t_merge_across_partitions_select_final.sql | 20 ++++++++++++---- 2 files changed, 31 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index cf8799052df..8b1e5989de5 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1335,13 +1335,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( data_settings->index_granularity, index_granularity_bytes); - const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( - settings.merge_tree_min_rows_for_concurrent_read, - settings.merge_tree_min_bytes_for_concurrent_read, - data_settings->index_granularity, - index_granularity_bytes, - sum_marks); - if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; @@ -1381,6 +1374,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( /// we will store lonely parts with level > 0 to use parallel select on them. std::vector lonely_parts; size_t total_rows_in_lonely_parts = 0; + size_t sum_marks_in_lonely_parts = 0; for (size_t range_index = 0; range_index < parts_to_merge_ranges.size() - 1; ++range_index) { @@ -1398,6 +1392,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( parts_to_merge_ranges[range_index]->data_part->info.level > 0) { total_rows_in_lonely_parts += parts_to_merge_ranges[range_index]->getRowsCount(); + sum_marks_in_lonely_parts += parts_to_merge_ranges[range_index]->getMarksCount(); lonely_parts.push_back(std::move(*parts_to_merge_ranges[range_index])); continue; } @@ -1485,9 +1480,21 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( size_t num_streams_for_lonely_parts = num_streams * lonely_parts.size(); + const size_t min_marks_for_concurrent_read = minMarksForConcurrentRead( + settings.merge_tree_min_rows_for_concurrent_read, + settings.merge_tree_min_bytes_for_concurrent_read, + data_settings->index_granularity, + index_granularity_bytes, + sum_marks_in_lonely_parts); + + /// Reduce the number of num_streams_for_lonely_parts if the data is small. + if (sum_marks_in_lonely_parts < num_streams_for_lonely_parts * min_marks_for_concurrent_read && lonely_parts.size() < num_streams_for_lonely_parts) + num_streams_for_lonely_parts = std::max((sum_marks_in_lonely_parts + min_marks_for_concurrent_read - 1) / min_marks_for_concurrent_read, lonely_parts.size()); + + MergeTreeReadPoolPtr pool = std::make_shared( num_streams_for_lonely_parts, - sum_marks, + sum_marks_in_lonely_parts, min_marks_for_concurrent_read, std::move(lonely_parts), data, diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql index 0e0f0325c7b..b50e47daa0c 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql @@ -1,5 +1,7 @@ DROP TABLE IF EXISTS select_final; +SET do_not_merge_across_partitions_select_final = 1; + CREATE TABLE select_final (t DateTime, x Int32, string String) ENGINE = ReplacingMergeTree() PARTITION BY toYYYYMM(t) ORDER BY (x, t); INSERT INTO select_final SELECT toDate('2000-01-01'), number, '' FROM numbers(2); @@ -9,7 +11,7 @@ INSERT INTO select_final SELECT toDate('2020-01-01'), number, '' FROM numbers(2) INSERT INTO select_final SELECT toDate('2020-01-01'), number + 1, '' FROM numbers(2); -SELECT * FROM select_final FINAL ORDER BY x SETTINGS do_not_merge_across_partitions_select_final = 1; +SELECT * FROM select_final FINAL ORDER BY x; TRUNCATE TABLE select_final; @@ -21,8 +23,18 @@ OPTIMIZE TABLE select_final FINAL; INSERT INTO select_final SELECT toDate('2020-01-01'), number, '' FROM numbers(2); INSERT INTO select_final SELECT toDate('2020-01-01'), number, 'updated' FROM numbers(2); -SELECT max(x) FROM select_final FINAL where string = 'updated' SETTINGS do_not_merge_across_partitions_select_final = 1; -SELECT arrayUniq(thread_ids) FROM system.query_log ORDER BY event_time LIMIT 1; +SELECT max(x) FROM select_final FINAL where string = 'updated'; + +TRUNCATE TABLE select_final; + +INSERT INTO select_final SELECT toDate('2000-01-01'), number, '' FROM numbers(500000); + +OPTIMIZE TABLE select_final FINAL; + +SELECT max(x) FROM select_final FINAL; + +SYSTEM FLUSH LOGS; + +SELECT length(thread_ids) > 1 FROM system.query_log WHERE query='SELECT max(x) FROM select_final FINAL;' AND type='QueryFinish' ORDER BY event_time DESC LIMIT 1; DROP TABLE select_final; - From 1fe003b85e6af635fd3359c16c77355ad1c71803 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 29 Jan 2021 22:09:34 +0300 Subject: [PATCH 39/94] Make system.asynchronous_metrics available just after start --- src/Interpreters/AsynchronousMetrics.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index 88c2221be76..f727b1d6b48 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -58,6 +58,9 @@ public: /// Separate method allows to initialize the `servers` variable beforehand. void start() { + /// Update once right now, to make metrics available just after server start + /// (without waiting for asynchronous_metrics_update_period_s). + update(); thread = std::make_unique([this] { run(); }); } From d3fe53a44fa4bc181bd853b63814dfff8b1841bc Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 29 Jan 2021 23:15:26 +0300 Subject: [PATCH 40/94] Update reference --- ...01524_do_not_merge_across_partitions_select_final.reference | 3 ++- .../01524_do_not_merge_across_partitions_select_final.sql | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference index 963359440c6..87eb40c57b0 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference @@ -5,4 +5,5 @@ 2000-01-01 00:00:00 2 2020-01-01 00:00:00 2 1 -16 +499999 +1 diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql index b50e47daa0c..5d20330014a 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql @@ -35,6 +35,6 @@ SELECT max(x) FROM select_final FINAL; SYSTEM FLUSH LOGS; -SELECT length(thread_ids) > 1 FROM system.query_log WHERE query='SELECT max(x) FROM select_final FINAL;' AND type='QueryFinish' ORDER BY event_time DESC LIMIT 1; +SELECT length(thread_ids) > 1 FROM system.query_log WHERE query='SELECT max(x) FROM select_final FINAL;' AND type='QueryFinish' AND current_database = currentDatabase() ORDER BY event_time DESC LIMIT 1; DROP TABLE select_final; From 2a3aaed562281ba40cb7290fb15aa12c4368f664 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sat, 30 Jan 2021 00:41:44 +0300 Subject: [PATCH 41/94] Update test --- .../01524_do_not_merge_across_partitions_select_final.reference | 2 +- .../01524_do_not_merge_across_partitions_select_final.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference index 87eb40c57b0..a3f2106cd5f 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.reference @@ -6,4 +6,4 @@ 2020-01-01 00:00:00 2 1 499999 -1 +5 diff --git a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql index 5d20330014a..25c47c008bd 100644 --- a/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql +++ b/tests/queries/0_stateless/01524_do_not_merge_across_partitions_select_final.sql @@ -35,6 +35,6 @@ SELECT max(x) FROM select_final FINAL; SYSTEM FLUSH LOGS; -SELECT length(thread_ids) > 1 FROM system.query_log WHERE query='SELECT max(x) FROM select_final FINAL;' AND type='QueryFinish' AND current_database = currentDatabase() ORDER BY event_time DESC LIMIT 1; +SELECT length(thread_ids) FROM system.query_log WHERE query='SELECT max(x) FROM select_final FINAL;' AND type='QueryFinish' AND current_database = currentDatabase() ORDER BY event_time DESC LIMIT 1; DROP TABLE select_final; From a9d9a8f3a98b1803c41e3643364b90d4b13c6d41 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 30 Jan 2021 01:21:51 +0300 Subject: [PATCH 42/94] Update Suggest.cpp --- programs/client/Suggest.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/programs/client/Suggest.cpp b/programs/client/Suggest.cpp index 297b371d987..dfa7048349e 100644 --- a/programs/client/Suggest.cpp +++ b/programs/client/Suggest.cpp @@ -87,10 +87,8 @@ Suggest::Suggest() void Suggest::loadImpl(Connection & connection, const ConnectionTimeouts & timeouts, size_t suggestion_limit) { - /// /// NOTE: Once you will update the completion list, /// do not forget to update 01676_clickhouse_client_autocomplete.sh - /// std::stringstream query; // STYLE_CHECK_ALLOW_STD_STRING_STREAM query << "SELECT DISTINCT arrayJoin(extractAll(name, '[\\\\w_]{2,}')) AS res FROM (" From c7fa78be1e590c8ecf0cfb56d151b8740a3f94ac Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Sat, 30 Jan 2021 16:32:07 +0300 Subject: [PATCH 43/94] Update docs/ru/sql-reference/statements/select/index.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/statements/select/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index 536511ed5e8..ab9d538f7fd 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -168,7 +168,7 @@ Code: 42. DB::Exception: Received from localhost:9000. DB::Exception: Number of ### APPLY {#apply-modifier} -Вызывает некоторую функцию для каждой строки, которая возвращает внешнее табличное выражение запроса. +Вызывает указанную функцию для каждой строки, возвращаемой внешним табличным выражением запроса. **Синтаксис:** From a46c8e0c73e07637144582b18d747271d5ba1703 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Sat, 30 Jan 2021 16:33:49 +0300 Subject: [PATCH 44/94] Update docs/ru/sql-reference/statements/select/index.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/statements/select/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index ab9d538f7fd..57fcc0244cc 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -192,7 +192,7 @@ SELECT * APPLY(sum) FROM columns_transformers; ### EXCEPT {#except-modifier} -Указывает имена одного или нескольких столбцов для исключения из результата. +Исключает из результата запроса один или несколько столбцов. **Синтаксис:** From df28433457947ec02f39aeccede9d0ee58008a9c Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Sat, 30 Jan 2021 16:34:39 +0300 Subject: [PATCH 45/94] Update docs/ru/sql-reference/statements/select/index.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/statements/select/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index 57fcc0244cc..b0b6e80d7be 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -215,7 +215,7 @@ SELECT * EXCEPT (i) from columns_transformers; ### REPLACE {#replace-modifier} -Указывает одно или несколько [выражений алиасов](../../../sql-reference/syntax.md#syntax-expression_aliases). Каждый алиас должен соответствовать имени столбца из запроса `SELECT *`. В списке столбцов из результата запроса имя столбца, соответствующее алиасу, заменяется выражением в модификаторе `REPLACE`. +Определяет одно или несколько [выражений алиасов](../../../sql-reference/syntax.md#syntax-expression_aliases). Каждый алиас должен соответствовать имени столбца из запроса `SELECT *`. В списке столбцов результата запроса имя столбца, соответствующее алиасу, заменяется выражением в модификаторе `REPLACE`. Этот модификатор не изменяет имена или порядок столбцов. Однако он может изменить значение и тип значения. From 978f23e3437d0d0eb64727e87ffe1b80289c9fc2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 31 Jan 2021 11:23:52 +0300 Subject: [PATCH 46/94] Fix SIGSEGV due to accessing GRPCServer::currentConnections() before initialization This PR makes asynchronous metrics available just after start, and this breaks GRPCServer, since it requires the server be started before accessing currentConnections(). However it is possible to trigger the same SIGSEGV even without that patch, with some timeouts during previous server initilizations and small asynchronous_metrics_update_period_s. Fix this by creating GRPCServer::Runner in the ctor. Stacktrace: 26.842505 [ 7 ] {} Application: Listening for MySQL compatibility protocol: 0.0.0.0:9004 26.842562 [ 7 ] {} Application: Listening for gRPC protocol: 0.0.0.0:9100 26.842600 [ 7 ] {} AsynchronousMetrics: MemoryTracking: was 350.72 KiB, peak 5.25 MiB, will set to 232.63 MiB (RSS), difference: 232.29 MiB 26.842834 [ 8 ] {} BaseDaemon: Received signal 11 26.843014 [ 39 ] {} BaseDaemon: ######################################## 26.843055 [ 39 ] {} BaseDaemon: (version 21.2.1.5858, build id: B27D5550AC34F9091BC4437D8021B752EDB34FBB) (from thread 7) (no query) Received signal Segmentation fault (11) 26.843085 [ 39 ] {} BaseDaemon: Address: 0x78 Access: read. Address not mapped to object. 26.843104 [ 39 ] {} BaseDaemon: Stack trace: 0x7fe8c4e6afc4 0x13683cf1 0xf52efe5 0xe7c9137 0x85596e1 0x8553635 0x11b7a313 0x8545ebc 0x8544b25 0x84e1cbe 0x7fe8c4c940b3 0x84ac22e 26.843136 [ 39 ] {} BaseDaemon: 2. pthread_mutex_lock @ 0xbfc4 in /usr/lib/x86_64-linux-gnu/libpthread-2.31.so 26.843162 [ 39 ] {} BaseDaemon: 3. std::__1::mutex::lock() @ 0x13683cf1 in ? 26.843188 [ 39 ] {} BaseDaemon: 4. DB::GRPCServer::currentConnections() const @ 0xf52efe5 in /usr/bin/clickhouse 26.843204 [ 39 ] {} BaseDaemon: 5. DB::AsynchronousMetrics::update() @ 0xe7c9137 in /usr/bin/clickhouse 26.843228 [ 39 ] {} BaseDaemon: 6. DB::AsynchronousMetrics::start() @ 0x85596e1 in /usr/bin/clickhouse 26.843248 [ 39 ] {} BaseDaemon: 7. DB::Server::main(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&) @ 0x8553635 in /usr/bin/clickhouse 26.843270 [ 39 ] {} BaseDaemon: 8. Poco::Util::Application::run() @ 0x11b7a313 in /usr/bin/clickhouse 26.843284 [ 39 ] {} BaseDaemon: 9. DB::Server::run() @ 0x8545ebc in /usr/bin/clickhouse 26.843299 [ 39 ] {} BaseDaemon: 10. mainEntryClickHouseServer(int, char**) @ 0x8544b25 in /usr/bin/clickhouse 26.843313 [ 39 ] {} BaseDaemon: 11. main @ 0x84e1cbe in /usr/bin/clickhouse 26.843331 [ 39 ] {} BaseDaemon: 12. __libc_start_main @ 0x270b3 in /usr/lib/x86_64-linux-gnu/libc-2.31.so 26.843346 [ 39 ] {} BaseDaemon: 13. _start @ 0x84ac22e in /usr/bin/clickhouse --- src/Server/GRPCServer.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 475bfc81801..c3492e9ea8a 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1613,7 +1613,10 @@ private: GRPCServer::GRPCServer(IServer & iserver_, const Poco::Net::SocketAddress & address_to_listen_) - : iserver(iserver_), address_to_listen(address_to_listen_), log(&Poco::Logger::get("GRPCServer")) + : iserver(iserver_) + , address_to_listen(address_to_listen_) + , log(&Poco::Logger::get("GRPCServer")) + , runner(std::make_unique(*this)) {} GRPCServer::~GRPCServer() @@ -1644,7 +1647,6 @@ void GRPCServer::start() queue = builder.AddCompletionQueue(); grpc_server = builder.BuildAndStart(); - runner = std::make_unique(*this); runner->start(); } From 11f144f5890984fac18d571fe9f3da6e70039214 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 Jan 2021 13:02:35 +0300 Subject: [PATCH 47/94] Fix deadlock in system.text_log --- src/Interpreters/SystemLog.h | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 6c56565a152..101bc752f43 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -229,9 +230,18 @@ void SystemLog::startup() } +static thread_local bool recursive_add_call = false; + template void SystemLog::add(const LogElement & element) { + /// It is possible that the method will be called recursively. + /// Better to drop these events to avoid complications. + if (recursive_add_call) + return; + recursive_add_call = true; + SCOPE_EXIT({ recursive_add_call = false; }); + /// Memory can be allocated while resizing on queue.push_back. /// The size of allocation can be in order of a few megabytes. /// But this should not be accounted for query memory usage. From 051e4832b998909ef47b6649a8548954583de3da Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 Jan 2021 13:05:00 +0300 Subject: [PATCH 48/94] Add a test --- .../queries/0_stateless/01683_text_log_deadlock.reference | 1 + tests/queries/0_stateless/01683_text_log_deadlock.sh | 7 +++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/01683_text_log_deadlock.reference create mode 100755 tests/queries/0_stateless/01683_text_log_deadlock.sh diff --git a/tests/queries/0_stateless/01683_text_log_deadlock.reference b/tests/queries/0_stateless/01683_text_log_deadlock.reference new file mode 100644 index 00000000000..b358a63ce24 --- /dev/null +++ b/tests/queries/0_stateless/01683_text_log_deadlock.reference @@ -0,0 +1 @@ +queries 100000 diff --git a/tests/queries/0_stateless/01683_text_log_deadlock.sh b/tests/queries/0_stateless/01683_text_log_deadlock.sh new file mode 100755 index 00000000000..ddb10c2d7f3 --- /dev/null +++ b/tests/queries/0_stateless/01683_text_log_deadlock.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_BENCHMARK --secure -i 100000 -c 32 --query 'SELECT 1' 2>&1 | grep -oF 'queries 100000' From ddff28e594d92ee6b8d0a495649720556232a785 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 Jan 2021 13:23:50 +0300 Subject: [PATCH 49/94] Fix UBSan report in intDiv --- src/Functions/DivisionUtils.h | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Functions/DivisionUtils.h b/src/Functions/DivisionUtils.h index 9f9cfc1e72c..97f5e81337c 100644 --- a/src/Functions/DivisionUtils.h +++ b/src/Functions/DivisionUtils.h @@ -3,8 +3,10 @@ #include #include #include +#include #include + #if !defined(ARCADIA_BUILD) # include #endif @@ -87,7 +89,12 @@ struct DivideIntegralImpl return static_cast(checkedDivision(static_cast(a), static_cast(b))); } else + { + if (!isFinite(a) || !isFinite(b)) + throw Exception("Cannot perform integer division on infinite floating point numbers", ErrorCodes::ILLEGAL_DIVISION); + return static_cast(checkedDivision(CastA(a), CastB(b))); + } } #if USE_EMBEDDED_COMPILER @@ -114,6 +121,9 @@ struct ModuloImpl } else { + if (!isFinite(a) || !isFinite(b)) + throw Exception("Cannot perform integer division on infinite floating point numbers", ErrorCodes::ILLEGAL_DIVISION); + throwIfDivisionLeadsToFPE(IntegerAType(a), IntegerBType(b)); if constexpr (is_big_int_v || is_big_int_v) From 4eb21ca39671b49d27b1a2c87387722d498fc510 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 31 Jan 2021 13:25:31 +0300 Subject: [PATCH 50/94] Add a test --- tests/queries/0_stateless/01683_intdiv_ubsan.reference | 0 tests/queries/0_stateless/01683_intdiv_ubsan.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/01683_intdiv_ubsan.reference create mode 100644 tests/queries/0_stateless/01683_intdiv_ubsan.sql diff --git a/tests/queries/0_stateless/01683_intdiv_ubsan.reference b/tests/queries/0_stateless/01683_intdiv_ubsan.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01683_intdiv_ubsan.sql b/tests/queries/0_stateless/01683_intdiv_ubsan.sql new file mode 100644 index 00000000000..adac2505745 --- /dev/null +++ b/tests/queries/0_stateless/01683_intdiv_ubsan.sql @@ -0,0 +1 @@ +SELECT DISTINCT intDiv(number, nan) FROM numbers(10); -- { serverError 153 } From cad9f2da6cedcb5273e4585890bf9c71211f6690 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sun, 31 Jan 2021 14:46:44 +0300 Subject: [PATCH 51/94] Update the system.distributed_ddl_queue system table MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил английскую версию и выполнил перевод на русский язык. --- .../system-tables/distributed_ddl_queue.md | 28 ++++---- .../system-tables/distributed_ddl_queue.md | 65 +++++++++++++++++++ 2 files changed, 78 insertions(+), 15 deletions(-) create mode 100644 docs/ru/operations/system-tables/distributed_ddl_queue.md diff --git a/docs/en/operations/system-tables/distributed_ddl_queue.md b/docs/en/operations/system-tables/distributed_ddl_queue.md index 643bdee6def..c252458af8a 100644 --- a/docs/en/operations/system-tables/distributed_ddl_queue.md +++ b/docs/en/operations/system-tables/distributed_ddl_queue.md @@ -1,22 +1,21 @@ # system.distributed_ddl_queue {#system_tables-distributed_ddl_queue} -Contains information about distributed ddl queries (ON CLUSTER queries) that were executed on a cluster. +Contains information about [distributed ddl queries (ON CLUSTER clause)](../../sql-reference/distributed-ddl.md) that were executed on a cluster. Columns: -- `entry` ([String](../../sql-reference/data-types/string.md)) - Query id. -- `host_name` ([String](../../sql-reference/data-types/string.md)) - Hostname. -- `host_address` ([String](../../sql-reference/data-types/string.md)) - IP address that the Hostname resolves to. -- `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) - Host Port. -- `status` ([Enum](../../sql-reference/data-types/enum.md)) - Stats of the query. -- `cluster` ([String](../../sql-reference/data-types/string.md)) - Cluster name. -- `query` ([String](../../sql-reference/data-types/string.md)) - Query executed. -- `initiator` ([String](../../sql-reference/data-types/string.md)) - Nod that executed the query. -- `query_start_time` ([Date](../../sql-reference/data-types/date.md)) — Query start time. -- `query_finish_time` ([Date](../../sql-reference/data-types/date.md)) — Query finish time. -- `query_duration_ms` ([UInt64](../../sql-reference/data-types/datetime64.md)) — Duration of query execution in milliseconds. -- `exception_code` ([Enum](../../sql-reference/data-types/enum.md)) - Exception code from ZooKeeper. - +- `entry` ([String](../../sql-reference/data-types/string.md)) — Query id. +- `host_name` ([String](../../sql-reference/data-types/string.md)) — Hostname. +- `host_address` ([String](../../sql-reference/data-types/string.md)) — IP address that the Hostname resolves to. +- `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — Host Port. +- `status` ([Enum8](../../sql-reference/data-types/enum.md)) — Status of the query. +- `cluster` ([String](../../sql-reference/data-types/string.md)) — Cluster name. +- `query` ([String](../../sql-reference/data-types/string.md)) — Query executed. +- `initiator` ([String](../../sql-reference/data-types/string.md)) — Node that executed the query. +- `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query start time. +- `query_finish_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query finish time. +- `query_duration_ms` ([UInt64](../../sql-reference/data-types/datetime64.md)) — Duration of query execution (in milliseconds). +- `exception_code` ([Enum8](../../sql-reference/data-types/enum.md)) — Exception code from [ZooKeeper](../../operations/tips.md#zookeeper). **Example** @@ -62,6 +61,5 @@ exception_code: ZOK 2 rows in set. Elapsed: 0.025 sec. ``` - [Original article](https://clickhouse.tech/docs/en/operations/system_tables/distributed_ddl_queuedistributed_ddl_queue.md) \ No newline at end of file diff --git a/docs/ru/operations/system-tables/distributed_ddl_queue.md b/docs/ru/operations/system-tables/distributed_ddl_queue.md new file mode 100644 index 00000000000..058ed06f639 --- /dev/null +++ b/docs/ru/operations/system-tables/distributed_ddl_queue.md @@ -0,0 +1,65 @@ +# system.distributed_ddl_queue {#system_tables-distributed_ddl_queue} + +Содержит информацию о [распределенных ddl запросах (секция ON CLUSTER)](../../sql-reference/distributed-ddl.md), которые были выполнены на кластере. + +Столбцы: + +- `entry` ([String](../../sql-reference/data-types/string.md)) — идентификатор запроса. +- `host_name` ([String](../../sql-reference/data-types/string.md)) — имя хоста. +- `host_address` ([String](../../sql-reference/data-types/string.md)) — IP-адрес хоста. +- `port` ([UInt16](../../sql-reference/data-types/int-uint.md)) — порт для соединения с сервером. +- `status` ([Enum8](../../sql-reference/data-types/enum.md)) — состояние запроса. +- `cluster` ([String](../../sql-reference/data-types/string.md)) — имя кластера. +- `query` ([String](../../sql-reference/data-types/string.md)) — выполненный запрос. +- `initiator` ([String](../../sql-reference/data-types/string.md)) — узел, выполнивший запрос. +- `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время начала запроса. +- `query_finish_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время окончания запроса. +- `query_duration_ms` ([UInt64](../../sql-reference/data-types/datetime64.md)) — продолжительность выполнения запроса (в миллисекундах). +- `exception_code` ([Enum8](../../sql-reference/data-types/enum.md)) — код исключения из [ZooKeeper](../../operations/tips.md#zookeeper). + +**Пример** + +``` sql +SELECT * +FROM system.distributed_ddl_queue +WHERE cluster = 'test_cluster' +LIMIT 2 +FORMAT Vertical + +Query id: f544e72a-6641-43f1-836b-24baa1c9632a + +Row 1: +────── +entry: query-0000000000 +host_name: clickhouse01 +host_address: 172.23.0.11 +port: 9000 +status: Finished +cluster: test_cluster +query: CREATE DATABASE test_db UUID '4a82697e-c85e-4e5b-a01e-a36f2a758456' ON CLUSTER test_cluster +initiator: clickhouse01:9000 +query_start_time: 2020-12-30 13:07:51 +query_finish_time: 2020-12-30 13:07:51 +query_duration_ms: 6 +exception_code: ZOK + +Row 2: +────── +entry: query-0000000000 +host_name: clickhouse02 +host_address: 172.23.0.12 +port: 9000 +status: Finished +cluster: test_cluster +query: CREATE DATABASE test_db UUID '4a82697e-c85e-4e5b-a01e-a36f2a758456' ON CLUSTER test_cluster +initiator: clickhouse01:9000 +query_start_time: 2020-12-30 13:07:51 +query_finish_time: 2020-12-30 13:07:51 +query_duration_ms: 6 +exception_code: ZOK + +2 rows in set. Elapsed: 0.025 sec. +``` + +[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/system_tables/distributed_ddl_queuedistributed_ddl_queue.md) + \ No newline at end of file From 1aa8d7fdfb79df5b98a1efab3586e1b69c78165c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 31 Jan 2021 21:49:27 +0300 Subject: [PATCH 52/94] BloomFilter index crash fix --- src/Interpreters/Set.cpp | 11 +++- ...681_bloom_filter_nullable_column.reference | 10 ++++ .../01681_bloom_filter_nullable_column.sql | 55 +++++++++++++++++++ 3 files changed, 75 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01681_bloom_filter_nullable_column.reference create mode 100644 tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index faec94de9ac..b150eabda65 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -140,7 +140,16 @@ void Set::setHeader(const Block & header) ConstNullMapPtr null_map{}; ColumnPtr null_map_holder; if (!transform_null_in) + { + /// We convert nullable columns to non nullable we also need to update nullable types + for (size_t i = 0; i < set_elements_types.size(); ++i) + { + data_types[i] = removeNullable(data_types[i]); + set_elements_types[i] = removeNullable(set_elements_types[i]); + } + extractNestedColumnsAndNullMap(key_columns, null_map); + } if (fill_set_elements) { @@ -182,7 +191,7 @@ bool Set::insertFromBlock(const Block & block) ConstNullMapPtr null_map{}; ColumnPtr null_map_holder; if (!transform_null_in) - null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); + null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); /// Filter to extract distinct values from the block. ColumnUInt8::MutablePtr filter; diff --git a/tests/queries/0_stateless/01681_bloom_filter_nullable_column.reference b/tests/queries/0_stateless/01681_bloom_filter_nullable_column.reference new file mode 100644 index 00000000000..2616e6c2a5c --- /dev/null +++ b/tests/queries/0_stateless/01681_bloom_filter_nullable_column.reference @@ -0,0 +1,10 @@ +NullableTuple with transform_null_in=0 +NullableTuple with transform_null_in=1 +NullableColumnFromCast with transform_null_in=0 +1 test +NullableColumnFromCast with transform_null_in=1 +1 test +NullableColumnFromTable with transform_null_in=0 +1 test +NullableColumnFromTable with transform_null_in=1 +1 test diff --git a/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql b/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql new file mode 100644 index 00000000000..d4ae56d14a6 --- /dev/null +++ b/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql @@ -0,0 +1,55 @@ +CREATE DATABASE 01681_bloom_filter_nullable_column; + +DROP TABLE IF EXISTS 01681_bloom_filter_nullable_column.bloom_filter_nullable_index; + +CREATE TABLE 01681_bloom_filter_nullable_column.bloom_filter_nullable_index + ( + order_key UInt64, + str Nullable(String), + + INDEX idx (str) TYPE bloom_filter GRANULARITY 1 + ) + ENGINE = MergeTree() + ORDER BY order_key SETTINGS index_granularity = 6; + +INSERT INTO 01681_bloom_filter_nullable_column.bloom_filter_nullable_index VALUES (1, 'test'); +INSERT INTO 01681_bloom_filter_nullable_column.bloom_filter_nullable_index VALUES (2, 'test2'); + +SELECT 'NullableTuple with transform_null_in=0'; +SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN + (SELECT '1048576', str FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index) SETTINGS transform_null_in = 0; +SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN + (SELECT '1048576', str FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index) SETTINGS transform_null_in = 0; + +SELECT 'NullableTuple with transform_null_in=1'; + +SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN + (SELECT '1048576', str FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index) SETTINGS transform_null_in = 1; -- { serverError 20 } + +SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN + (SELECT '1048576', str FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index) SETTINGS transform_null_in = 1; -- { serverError 20 } + + +SELECT 'NullableColumnFromCast with transform_null_in=0'; +SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN + (SELECT cast('test', 'Nullable(String)')) SETTINGS transform_null_in = 0; + +SELECT 'NullableColumnFromCast with transform_null_in=1'; +SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN + (SELECT cast('test', 'Nullable(String)')) SETTINGS transform_null_in = 1; + +CREATE TABLE 01681_bloom_filter_nullable_column.nullable_string_value (value Nullable(String)) ENGINE=TinyLog; +INSERT INTO 01681_bloom_filter_nullable_column.nullable_string_value VALUES ('test'); + +SELECT 'NullableColumnFromTable with transform_null_in=0'; +SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN + (SELECT value FROM 01681_bloom_filter_nullable_column.nullable_string_value) SETTINGS transform_null_in = 0; + +SELECT 'NullableColumnFromTable with transform_null_in=1'; +SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN + (SELECT value FROM 01681_bloom_filter_nullable_column.nullable_string_value) SETTINGS transform_null_in = 1; + +DROP TABLE 01681_bloom_filter_nullable_column.nullable_string_value; + +DROP TABLE 01681_bloom_filter_nullable_column.bloom_filter_nullable_index; +DROP DATABASE 01681_bloom_filter_nullable_column; From 65183f1feae3fe017058b5658c8d476e0497e962 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 31 Jan 2021 23:54:52 +0300 Subject: [PATCH 53/94] Fixed tests --- tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql b/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql index d4ae56d14a6..2a61d66ac73 100644 --- a/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql +++ b/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql @@ -1,3 +1,4 @@ +DROP DATABASE IF EXISTS 01681_bloom_filter_nullable_column; CREATE DATABASE 01681_bloom_filter_nullable_column; DROP TABLE IF EXISTS 01681_bloom_filter_nullable_column.bloom_filter_nullable_index; From 84d3d6e7280907a22f6a2ce86b9bf3040b4871f1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 1 Feb 2021 02:10:41 +0300 Subject: [PATCH 54/94] fix 'Unmatched parentheses', add test --- src/Parsers/TokenIterator.cpp | 7 ++-- src/Parsers/TokenIterator.h | 2 +- src/Parsers/parseQuery.cpp | 2 +- .../01180_client_syntax_errors.expect | 32 +++++++++++++++++++ .../01180_client_syntax_errors.reference | 0 5 files changed, 38 insertions(+), 5 deletions(-) create mode 100755 tests/queries/0_stateless/01180_client_syntax_errors.expect create mode 100644 tests/queries/0_stateless/01180_client_syntax_errors.reference diff --git a/src/Parsers/TokenIterator.cpp b/src/Parsers/TokenIterator.cpp index 18360ed29ae..08877e0b2fe 100644 --- a/src/Parsers/TokenIterator.cpp +++ b/src/Parsers/TokenIterator.cpp @@ -4,13 +4,14 @@ namespace DB { -UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin, Token last) +UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin) { /// We have just two kind of parentheses: () and []. UnmatchedParentheses stack; - for (TokenIterator it = begin; - it.isValid() && it->begin <= last.begin; ++it) + /// We have to iterate through all tokens until the end to avoid false positive "Unmatched parentheses" error + /// when parser failed in the middle of the query. + for (TokenIterator it = begin; it.isValid(); ++it) { if (it->type == TokenType::OpeningRoundBracket || it->type == TokenType::OpeningSquareBracket) { diff --git a/src/Parsers/TokenIterator.h b/src/Parsers/TokenIterator.h index a95465500e0..e3a5b9f79c3 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -80,6 +80,6 @@ public: /// Returns positions of unmatched parentheses. using UnmatchedParentheses = std::vector; -UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin, Token last); +UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin); } diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index 650c0e40c8c..4309ae8955a 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -290,7 +290,7 @@ ASTPtr tryParseQuery( } /// Unmatched parentheses - UnmatchedParentheses unmatched_parens = checkUnmatchedParentheses(TokenIterator(tokens), last_token); + UnmatchedParentheses unmatched_parens = checkUnmatchedParentheses(TokenIterator(tokens)); if (!unmatched_parens.empty()) { out_error_message = getUnmatchedParenthesesErrorMessage(query_begin, diff --git a/tests/queries/0_stateless/01180_client_syntax_errors.expect b/tests/queries/0_stateless/01180_client_syntax_errors.expect new file mode 100755 index 00000000000..bc775ce2c57 --- /dev/null +++ b/tests/queries/0_stateless/01180_client_syntax_errors.expect @@ -0,0 +1,32 @@ +#!/usr/bin/expect -f + +log_user 0 +set timeout 5 +match_max 100000 +# A default timeout action is to do nothing, change it to fail +expect_after { + timeout { + exit 1 + } +} + +set basedir [file dirname $argv0] +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT" +expect ":) " + +# Make a query with syntax error +send -- "select \r" +expect "Syntax error: failed at position 7 (end of query):" +expect "Expected one of: " + +# Make another query with syntax error +send -- "CREATE TABLE t4 UUID '57f27aa5-141c-47c5-888a-9563681717f5' AS t1 (`rowNumberInAllBlocks()` UInt64, `toLowCardinality(arrayJoin(\['exchange', 'tables'\]))` LowCardinality(String)) ENGINE = MergeTree \r" +expect "Syntax error: failed at position 93 ('UInt64'):*" + +# Make a query with unmatched parentheses +send -- "select (1, 2\r" +expect "Syntax error: failed at position 8 ('('):" +expect "Unmatched parentheses: (" + +send -- "\4" +expect eof diff --git a/tests/queries/0_stateless/01180_client_syntax_errors.reference b/tests/queries/0_stateless/01180_client_syntax_errors.reference new file mode 100644 index 00000000000..e69de29bb2d From efb9293587e90daa786a24ca6a1055a2d59cd8f9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 1 Feb 2021 05:28:18 +0300 Subject: [PATCH 55/94] Adjust test scale --- tests/queries/0_stateless/01683_text_log_deadlock.reference | 2 +- tests/queries/0_stateless/01683_text_log_deadlock.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01683_text_log_deadlock.reference b/tests/queries/0_stateless/01683_text_log_deadlock.reference index b358a63ce24..76de47c4a46 100644 --- a/tests/queries/0_stateless/01683_text_log_deadlock.reference +++ b/tests/queries/0_stateless/01683_text_log_deadlock.reference @@ -1 +1 @@ -queries 100000 +queries 25000 diff --git a/tests/queries/0_stateless/01683_text_log_deadlock.sh b/tests/queries/0_stateless/01683_text_log_deadlock.sh index ddb10c2d7f3..ee772bffa27 100755 --- a/tests/queries/0_stateless/01683_text_log_deadlock.sh +++ b/tests/queries/0_stateless/01683_text_log_deadlock.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_BENCHMARK --secure -i 100000 -c 32 --query 'SELECT 1' 2>&1 | grep -oF 'queries 100000' +$CLICKHOUSE_BENCHMARK --secure -i 25000 -c 32 --query 'SELECT 1' 2>&1 | grep -oF 'queries 25000' From 768e461d28719bb6af2eecc32059e64429c4b6af Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Mon, 1 Feb 2021 10:40:48 +0800 Subject: [PATCH 56/94] add functional test --- src/Interpreters/DDLWorker.cpp | 6 ++-- .../01671_ddl_hang_timeout.reference | 0 .../0_stateless/01671_ddl_hang_timeout.sh | 28 +++++++++++++++++++ 3 files changed, 31 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01671_ddl_hang_timeout.reference create mode 100644 tests/queries/0_stateless/01671_ddl_hang_timeout.sh diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 85445eb0cff..e630c6abeb5 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -871,10 +871,10 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica( // Should return as soon as possible if the table is dropped. bool replica_dropped = replicated_storage->is_dropped; bool all_replicas_likely_detached = status.active_replicas == 0 && !DatabaseCatalog::instance().isTableExist(replicated_storage->getStorageID(), context); - if (replica_dropped || all_replicas_likely_detached) + if (replica_dropped || all_replicas_likely_detached) { - LOG_WARNING(log, "Table is dropped or detached permantly, task {} will not be executed.", task.entry_name); - task.execution_status = ExecutionStatus(ErrorCodes::UNFINISHED, "Cannot execute replicated DDL query, table is dropped or detached permantly"); + LOG_WARNING(log, ", task {} will not be executed.", task.entry_name); + task.execution_status = ExecutionStatus(ErrorCodes::UNFINISHED, "Cannot execute replicated DDL query, table is dropped or detached permanently"); return false; } diff --git a/tests/queries/0_stateless/01671_ddl_hang_timeout.reference b/tests/queries/0_stateless/01671_ddl_hang_timeout.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01671_ddl_hang_timeout.sh b/tests/queries/0_stateless/01671_ddl_hang_timeout.sh new file mode 100644 index 00000000000..3ea65075443 --- /dev/null +++ b/tests/queries/0_stateless/01671_ddl_hang_timeout.sh @@ -0,0 +1,28 @@ +#!/usr/bin/env bash +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../shell_config.sh +$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS test_01671" +$CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01671" +function thread_create_drop_table { + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT --query "CREATE TABLE test_01671.t1 (x UInt64, s Array(Nullable(String))) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01671/test_01671', 'r_$REPLICA') order by x" + sleep 0.0$RANDOM + $CLICKHOUSE_CLIENT --query "DROP TABLE test_01671.t1" + done +} +function thread_alter_table { + while true; do + $CLICKHOUSE_CLIENT --query "ALTER TABLE test_01671.t1 on cluster test_shard_localhost ADD COLUMN newcol UInt32" + sleep 0.0$RANDOM + done +} +export -f thread_create_drop_table +export -f thread_alter_table +timeout 20 bash -c "thread_create_drop_table" & +timeout 20 bash -c 'thread_alter_table' & +wait +sleep 1 + +$CLICKHOUSE_CLIENT --query "DROP DATABASE test_01671"; \ No newline at end of file From bef5af3f57986a103bc6510ee98ef0f981cb821d Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Mon, 1 Feb 2021 11:35:02 +0800 Subject: [PATCH 57/94] fix functional test --- tests/queries/0_stateless/01671_ddl_hang_timeout.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/01671_ddl_hang_timeout.sh diff --git a/tests/queries/0_stateless/01671_ddl_hang_timeout.sh b/tests/queries/0_stateless/01671_ddl_hang_timeout.sh old mode 100644 new mode 100755 index 3ea65075443..ec59b03b6d2 --- a/tests/queries/0_stateless/01671_ddl_hang_timeout.sh +++ b/tests/queries/0_stateless/01671_ddl_hang_timeout.sh @@ -7,14 +7,14 @@ $CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01671" function thread_create_drop_table { while true; do REPLICA=$(($RANDOM % 10)) - $CLICKHOUSE_CLIENT --query "CREATE TABLE test_01671.t1 (x UInt64, s Array(Nullable(String))) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01671/test_01671', 'r_$REPLICA') order by x" + $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS test_01671.t1 (x UInt64, s Array(Nullable(String))) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01671/test_01671', 'r_$REPLICA') order by x" 2>/dev/null sleep 0.0$RANDOM - $CLICKHOUSE_CLIENT --query "DROP TABLE test_01671.t1" + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_01671.t1" done } function thread_alter_table { while true; do - $CLICKHOUSE_CLIENT --query "ALTER TABLE test_01671.t1 on cluster test_shard_localhost ADD COLUMN newcol UInt32" + $CLICKHOUSE_CLIENT --query "ALTER TABLE test_01671.t1 on cluster test_shard_localhost ADD COLUMN newcol UInt32" >/dev/null 2>&1 sleep 0.0$RANDOM done } @@ -25,4 +25,4 @@ timeout 20 bash -c 'thread_alter_table' & wait sleep 1 -$CLICKHOUSE_CLIENT --query "DROP DATABASE test_01671"; \ No newline at end of file +$CLICKHOUSE_CLIENT --query "DROP DATABASE test_01671"; From e9a61c47380ad14c4b57cefd17758a092a9de2c3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 1 Feb 2021 07:52:18 +0300 Subject: [PATCH 58/94] More correct --- src/Functions/DivisionUtils.h | 22 ++++++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/src/Functions/DivisionUtils.h b/src/Functions/DivisionUtils.h index 97f5e81337c..18a0331fd3f 100644 --- a/src/Functions/DivisionUtils.h +++ b/src/Functions/DivisionUtils.h @@ -90,8 +90,15 @@ struct DivideIntegralImpl } else { - if (!isFinite(a) || !isFinite(b)) - throw Exception("Cannot perform integer division on infinite floating point numbers", ErrorCodes::ILLEGAL_DIVISION); + if constexpr (std::is_floating_point_v) + if (!isNaN(a) || a > std::numeric_limits::max() || a < std::numeric_limits::lowest()) + throw Exception("Cannot perform integer division on infinite or too large floating point numbers", + ErrorCodes::ILLEGAL_DIVISION); + + if constexpr (std::is_floating_point_v) + if (!isNaN(b) || b > std::numeric_limits::max() || b < std::numeric_limits::lowest()) + throw Exception("Cannot perform integer division on infinite or too large floating point numbers", + ErrorCodes::ILLEGAL_DIVISION); return static_cast(checkedDivision(CastA(a), CastB(b))); } @@ -121,8 +128,15 @@ struct ModuloImpl } else { - if (!isFinite(a) || !isFinite(b)) - throw Exception("Cannot perform integer division on infinite floating point numbers", ErrorCodes::ILLEGAL_DIVISION); + if constexpr (std::is_floating_point_v) + if (!isNaN(a) || a > std::numeric_limits::max() || a < std::numeric_limits::lowest()) + throw Exception("Cannot perform integer division on infinite or too large floating point numbers", + ErrorCodes::ILLEGAL_DIVISION); + + if constexpr (std::is_floating_point_v) + if (!isNaN(b) || b > std::numeric_limits::max() || b < std::numeric_limits::lowest()) + throw Exception("Cannot perform integer division on infinite or too large floating point numbers", + ErrorCodes::ILLEGAL_DIVISION); throwIfDivisionLeadsToFPE(IntegerAType(a), IntegerBType(b)); From 2459777f57a265378e949d7ab9926c29fc7b4de6 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 1 Feb 2021 07:55:58 +0300 Subject: [PATCH 59/94] Update 01681_bloom_filter_nullable_column.sql --- .../01681_bloom_filter_nullable_column.sql | 51 +++++++++---------- 1 file changed, 23 insertions(+), 28 deletions(-) diff --git a/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql b/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql index 2a61d66ac73..4af1f74fca6 100644 --- a/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql +++ b/tests/queries/0_stateless/01681_bloom_filter_nullable_column.sql @@ -1,9 +1,5 @@ -DROP DATABASE IF EXISTS 01681_bloom_filter_nullable_column; -CREATE DATABASE 01681_bloom_filter_nullable_column; - -DROP TABLE IF EXISTS 01681_bloom_filter_nullable_column.bloom_filter_nullable_index; - -CREATE TABLE 01681_bloom_filter_nullable_column.bloom_filter_nullable_index +DROP TABLE IF EXISTS bloom_filter_nullable_index; +CREATE TABLE bloom_filter_nullable_index ( order_key UInt64, str Nullable(String), @@ -13,44 +9,43 @@ CREATE TABLE 01681_bloom_filter_nullable_column.bloom_filter_nullable_index ENGINE = MergeTree() ORDER BY order_key SETTINGS index_granularity = 6; -INSERT INTO 01681_bloom_filter_nullable_column.bloom_filter_nullable_index VALUES (1, 'test'); -INSERT INTO 01681_bloom_filter_nullable_column.bloom_filter_nullable_index VALUES (2, 'test2'); +INSERT INTO bloom_filter_nullable_index VALUES (1, 'test'); +INSERT INTO bloom_filter_nullable_index VALUES (2, 'test2'); SELECT 'NullableTuple with transform_null_in=0'; -SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN - (SELECT '1048576', str FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index) SETTINGS transform_null_in = 0; -SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN - (SELECT '1048576', str FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index) SETTINGS transform_null_in = 0; +SELECT * FROM bloom_filter_nullable_index WHERE str IN + (SELECT '1048576', str FROM bloom_filter_nullable_index) SETTINGS transform_null_in = 0; +SELECT * FROM bloom_filter_nullable_index WHERE str IN + (SELECT '1048576', str FROM bloom_filter_nullable_index) SETTINGS transform_null_in = 0; SELECT 'NullableTuple with transform_null_in=1'; -SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN - (SELECT '1048576', str FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index) SETTINGS transform_null_in = 1; -- { serverError 20 } +SELECT * FROM bloom_filter_nullable_index WHERE str IN + (SELECT '1048576', str FROM bloom_filter_nullable_index) SETTINGS transform_null_in = 1; -- { serverError 20 } -SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN - (SELECT '1048576', str FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index) SETTINGS transform_null_in = 1; -- { serverError 20 } +SELECT * FROM bloom_filter_nullable_index WHERE str IN + (SELECT '1048576', str FROM bloom_filter_nullable_index) SETTINGS transform_null_in = 1; -- { serverError 20 } SELECT 'NullableColumnFromCast with transform_null_in=0'; -SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN +SELECT * FROM bloom_filter_nullable_index WHERE str IN (SELECT cast('test', 'Nullable(String)')) SETTINGS transform_null_in = 0; SELECT 'NullableColumnFromCast with transform_null_in=1'; -SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN +SELECT * FROM bloom_filter_nullable_index WHERE str IN (SELECT cast('test', 'Nullable(String)')) SETTINGS transform_null_in = 1; -CREATE TABLE 01681_bloom_filter_nullable_column.nullable_string_value (value Nullable(String)) ENGINE=TinyLog; -INSERT INTO 01681_bloom_filter_nullable_column.nullable_string_value VALUES ('test'); +DROP TABLE IF EXISTS nullable_string_value; +CREATE TABLE nullable_string_value (value Nullable(String)) ENGINE=TinyLog; +INSERT INTO nullable_string_value VALUES ('test'); SELECT 'NullableColumnFromTable with transform_null_in=0'; -SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN - (SELECT value FROM 01681_bloom_filter_nullable_column.nullable_string_value) SETTINGS transform_null_in = 0; +SELECT * FROM bloom_filter_nullable_index WHERE str IN + (SELECT value FROM nullable_string_value) SETTINGS transform_null_in = 0; SELECT 'NullableColumnFromTable with transform_null_in=1'; -SELECT * FROM 01681_bloom_filter_nullable_column.bloom_filter_nullable_index WHERE str IN - (SELECT value FROM 01681_bloom_filter_nullable_column.nullable_string_value) SETTINGS transform_null_in = 1; +SELECT * FROM bloom_filter_nullable_index WHERE str IN + (SELECT value FROM nullable_string_value) SETTINGS transform_null_in = 1; -DROP TABLE 01681_bloom_filter_nullable_column.nullable_string_value; - -DROP TABLE 01681_bloom_filter_nullable_column.bloom_filter_nullable_index; -DROP DATABASE 01681_bloom_filter_nullable_column; +DROP TABLE nullable_string_value; +DROP TABLE bloom_filter_nullable_index; From fa03fbdc5bb9ffc2f797b1bab6d3403638b87128 Mon Sep 17 00:00:00 2001 From: yiguolei <676222867@qq.com> Date: Mon, 1 Feb 2021 13:16:54 +0800 Subject: [PATCH 60/94] fix functional test --- tests/queries/0_stateless/01671_ddl_hang_timeout.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01671_ddl_hang_timeout.sh b/tests/queries/0_stateless/01671_ddl_hang_timeout.sh index ec59b03b6d2..fb4926c8b00 100755 --- a/tests/queries/0_stateless/01671_ddl_hang_timeout.sh +++ b/tests/queries/0_stateless/01671_ddl_hang_timeout.sh @@ -1,6 +1,6 @@ #!/usr/bin/env bash -CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=fatal CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh $CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS test_01671" $CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01671" From 1517decf4af597257e0b3b74fbaf7573c1be3475 Mon Sep 17 00:00:00 2001 From: "Mc.Spring" Date: Mon, 1 Feb 2021 13:43:39 +0800 Subject: [PATCH 61/94] fix cmake command issue add missing space for cmake command line --- docs/en/development/build-osx.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/build-osx.md b/docs/en/development/build-osx.md index 60365ad744a..e0b1be710f1 100644 --- a/docs/en/development/build-osx.md +++ b/docs/en/development/build-osx.md @@ -40,7 +40,7 @@ $ cd ClickHouse ``` bash $ mkdir build $ cd build -$ cmake ..-DCMAKE_C_COMPILER=`brew --prefix llvm`/bin/clang -DCMAKE_CXX_COMPILER=`brew --prefix llvm`/bin/clang++ -DCMAKE_PREFIX_PATH=`brew --prefix llvm` +$ cmake .. -DCMAKE_C_COMPILER=`brew --prefix llvm`/bin/clang -DCMAKE_CXX_COMPILER=`brew --prefix llvm`/bin/clang++ -DCMAKE_PREFIX_PATH=`brew --prefix llvm` $ ninja $ cd .. ``` From 57664bffe0be9e3dcc76b02aeefca9e5b8e44453 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 1 Feb 2021 10:11:41 +0300 Subject: [PATCH 62/94] Amend --- src/Functions/DivisionUtils.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Functions/DivisionUtils.h b/src/Functions/DivisionUtils.h index 18a0331fd3f..d0df7e41af1 100644 --- a/src/Functions/DivisionUtils.h +++ b/src/Functions/DivisionUtils.h @@ -91,12 +91,12 @@ struct DivideIntegralImpl else { if constexpr (std::is_floating_point_v) - if (!isNaN(a) || a > std::numeric_limits::max() || a < std::numeric_limits::lowest()) + if (isNaN(a) || a > std::numeric_limits::max() || a < std::numeric_limits::lowest()) throw Exception("Cannot perform integer division on infinite or too large floating point numbers", ErrorCodes::ILLEGAL_DIVISION); if constexpr (std::is_floating_point_v) - if (!isNaN(b) || b > std::numeric_limits::max() || b < std::numeric_limits::lowest()) + if (isNaN(b) || b > std::numeric_limits::max() || b < std::numeric_limits::lowest()) throw Exception("Cannot perform integer division on infinite or too large floating point numbers", ErrorCodes::ILLEGAL_DIVISION); @@ -129,12 +129,12 @@ struct ModuloImpl else { if constexpr (std::is_floating_point_v) - if (!isNaN(a) || a > std::numeric_limits::max() || a < std::numeric_limits::lowest()) + if (isNaN(a) || a > std::numeric_limits::max() || a < std::numeric_limits::lowest()) throw Exception("Cannot perform integer division on infinite or too large floating point numbers", ErrorCodes::ILLEGAL_DIVISION); if constexpr (std::is_floating_point_v) - if (!isNaN(b) || b > std::numeric_limits::max() || b < std::numeric_limits::lowest()) + if (isNaN(b) || b > std::numeric_limits::max() || b < std::numeric_limits::lowest()) throw Exception("Cannot perform integer division on infinite or too large floating point numbers", ErrorCodes::ILLEGAL_DIVISION); From bedc472bb0f7555f57c524cadec9288f0add76ee Mon Sep 17 00:00:00 2001 From: Stig Bakken Date: Thu, 28 Jan 2021 20:26:47 +0800 Subject: [PATCH 63/94] Show MaterializeMySQL tables in system.parts --- src/Storages/System/StorageSystemPartsBase.cpp | 8 ++++++++ .../materialize_with_ddl.py | 15 ++++++++++++++- .../test_materialize_mysql_database/test.py | 4 ++++ 3 files changed, 26 insertions(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 5a3d7027dd4..9b5bf5a4b48 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -119,6 +120,13 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const String engine_name = storage->getName(); +#if USE_MYSQL + if (auto * proxy = dynamic_cast(storage.get())) + { + auto nested = proxy->getNested(); + storage.swap(nested); + } +#endif if (!dynamic_cast(storage.get())) continue; diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 11600359b66..e3ac7fc0c14 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -733,7 +733,6 @@ def clickhouse_killed_while_insert(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE kill_clickhouse_while_insert") clickhouse_node.query("DROP DATABASE kill_clickhouse_while_insert") - def utf8mb4_test(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE IF EXISTS utf8mb4_test") clickhouse_node.query("DROP DATABASE IF EXISTS utf8mb4_test") @@ -743,3 +742,17 @@ def utf8mb4_test(clickhouse_node, mysql_node, service_name): clickhouse_node.query("CREATE DATABASE utf8mb4_test ENGINE = MaterializeMySQL('{}:3306', 'utf8mb4_test', 'root', 'clickhouse')".format(service_name)) check_query(clickhouse_node, "SELECT id, name FROM utf8mb4_test.test ORDER BY id", "1\t\U0001F984\n2\t\u2601\n") +def system_parts_test(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS system_parts_test") + clickhouse_node.query("DROP DATABASE IF EXISTS system_parts_test") + mysql_node.query("CREATE DATABASE system_parts_test") + mysql_node.query("CREATE TABLE system_parts_test.test ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") + mysql_node.query("INSERT INTO system_parts_test.test VALUES(1),(2),(3)") + def check_active_parts(num): + check_query(clickhouse_node, "SELECT count() FROM system.parts WHERE database = 'system_parts_test' AND table = 'test' AND active = 1", "{}\n".format(num)) + clickhouse_node.query("CREATE DATABASE system_parts_test ENGINE = MaterializeMySQL('{}:3306', 'system_parts_test', 'root', 'clickhouse')".format(service_name)) + check_active_parts(1) + mysql_node.query("INSERT INTO system_parts_test.test VALUES(4),(5),(6)") + check_active_parts(2) + clickhouse_node.query("OPTIMIZE TABLE system_parts_test.test") + check_active_parts(1) diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 22322671f13..8cd2f7def07 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -233,3 +233,7 @@ def test_clickhouse_killed_while_insert_8_0(started_cluster, started_mysql_8_0, def test_utf8mb4(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_5_7, "mysql1") materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + +@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) +def test_system_parts_table(started_cluster, started_mysql_8_0, clickhouse_node): + materialize_with_ddl.system_parts_test(clickhouse_node, started_mysql_8_0, "mysql8_0") From 4f955be8328350dc2eb25fd16f6cb13bb074395a Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 1 Feb 2021 12:32:36 +0300 Subject: [PATCH 64/94] fix sync --- src/Storages/StorageMaterializeMySQL.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Storages/StorageMaterializeMySQL.h b/src/Storages/StorageMaterializeMySQL.h index ea90c1ffc9e..f787470e2d2 100644 --- a/src/Storages/StorageMaterializeMySQL.h +++ b/src/Storages/StorageMaterializeMySQL.h @@ -1,6 +1,8 @@ #pragma once +#if !defined(ARCADIA_BUILD) #include "config_core.h" +#endif #if USE_MYSQL From dafb0ef4e99a24b6494a2e6cbcdef1458836c651 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 26 Jan 2021 23:49:52 +0300 Subject: [PATCH 65/94] ExecutableDictionarySource added implicit_key option --- src/Core/Block.cpp | 9 ++ src/Core/Block.h | 1 + src/Dictionaries/DictionarySourceHelpers.cpp | 45 +++--- src/Dictionaries/DictionarySourceHelpers.h | 16 +- src/Dictionaries/DictionaryStructure.cpp | 15 ++ src/Dictionaries/DictionaryStructure.h | 2 + .../ExecutableDictionarySource.cpp | 119 +++++++++++++- src/Dictionaries/ExecutableDictionarySource.h | 2 +- src/Dictionaries/HTTPDictionarySource.cpp | 12 +- tests/config/executable_dictionary.xml | 148 ++++++++++++++++++ .../01474_executable_dictionary.reference | 5 + .../01474_executable_dictionary.sql | 8 + 12 files changed, 347 insertions(+), 35 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index cd2855739e2..56c045bc8df 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -409,6 +409,15 @@ Block Block::cloneWithoutColumns() const return res; } +Block Block::cloneWithCutColumns(size_t start, size_t length) const +{ + Block copy = *this; + + for (size_t i = 0; i < copy.data.size(); ++i) + copy.data[i].column = copy.data[i].column->cut(start, length); + + return copy; +} Block Block::sortColumns() const { diff --git a/src/Core/Block.h b/src/Core/Block.h index f588373aaed..14f4f57caed 100644 --- a/src/Core/Block.h +++ b/src/Core/Block.h @@ -129,6 +129,7 @@ public: void setColumns(const Columns & columns); Block cloneWithColumns(const Columns & columns) const; Block cloneWithoutColumns() const; + Block cloneWithCutColumns(size_t start, size_t length) const; /** Get empty columns with the same types as in block. */ MutableColumns cloneEmptyColumns() const; diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index 309bc64e179..3227704ff4b 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -1,6 +1,5 @@ #include "DictionarySourceHelpers.h" #include -#include #include #include #include @@ -13,44 +12,54 @@ namespace DB { -/// For simple key -void formatIDs(BlockOutputStreamPtr & out, const std::vector & ids) + +void formatWithBlock(BlockOutputStreamPtr & out, Block block) { - auto column = ColumnUInt64::create(ids.size()); - memcpy(column->getData().data(), ids.data(), ids.size() * sizeof(ids.front())); - - Block block{{std::move(column), std::make_shared(), "id"}}; - out->writePrefix(); out->write(block); out->writeSuffix(); out->flush(); } +/// For simple key + +Block blockForIds(const std::vector & ids) +{ + auto column = ColumnUInt64::create(ids.size()); + memcpy(column->getData().data(), ids.data(), ids.size() * sizeof(ids.front())); + + Block block{{std::move(column), std::make_shared(), "id"}}; + + std::cerr << "Block for IDs size " << ids.size() << std::endl; + + return block; +} + /// For composite key -void formatKeys( + +Block blockForKeys( const DictionaryStructure & dict_struct, - BlockOutputStreamPtr & out, const Columns & key_columns, const std::vector & requested_rows) { Block block; + for (size_t i = 0, size = key_columns.size(); i < size; ++i) { const ColumnPtr & source_column = key_columns[i]; - auto filtered_column = source_column->cloneEmpty(); - filtered_column->reserve(requested_rows.size()); + size_t column_rows_size = source_column->size(); + + PaddedPODArray filter(column_rows_size, false); for (size_t idx : requested_rows) - filtered_column->insertFrom(*source_column, idx); + filter[idx] = true; - block.insert({std::move(filtered_column), (*dict_struct.key)[i].type, toString(i)}); + auto filtered_column = source_column->filter(filter, requested_rows.size()); + + block.insert({std::move(filtered_column), (*dict_struct.key)[i].type, (*dict_struct.key)[i].name}); } - out->writePrefix(); - out->write(block); - out->writeSuffix(); - out->flush(); + return block; } Context copyContextAndApplySettings( diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index 3f42700d336..d6b0dab8d4f 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -1,11 +1,15 @@ #pragma once #include -#include + #include + #include #include +#include +#include + namespace DB { class IBlockOutputStream; @@ -16,13 +20,16 @@ class Context; /// Write keys to block output stream. +void formatWithBlock(BlockOutputStreamPtr & out, Block block); + /// For simple key -void formatIDs(BlockOutputStreamPtr & out, const std::vector & ids); + +Block blockForIds(const std::vector & ids); /// For composite key -void formatKeys( + +Block blockForKeys( const DictionaryStructure & dict_struct, - BlockOutputStreamPtr & out, const Columns & key_columns, const std::vector & requested_rows); @@ -36,4 +43,5 @@ void applySettingsToContext( const std::string & config_prefix, Context & context, const Poco::Util::AbstractConfiguration & config); + } diff --git a/src/Dictionaries/DictionaryStructure.cpp b/src/Dictionaries/DictionaryStructure.cpp index 95c2e0a3e09..42a7cb6e4ec 100644 --- a/src/Dictionaries/DictionaryStructure.cpp +++ b/src/Dictionaries/DictionaryStructure.cpp @@ -281,6 +281,21 @@ size_t DictionaryStructure::getKeySize() const }); } +Strings DictionaryStructure::getKeysNames() const +{ + if (id) + return { id->name }; + + auto & key_attributes = *key; + + Strings keys_names; + keys_names.reserve(key_attributes.size()); + + for (const auto & key_attribute : key_attributes) + keys_names.emplace_back(key_attribute.name); + + return keys_names; +} static void checkAttributeKeys(const Poco::Util::AbstractConfiguration::Keys & keys) { diff --git a/src/Dictionaries/DictionaryStructure.h b/src/Dictionaries/DictionaryStructure.h index 945e1c55494..c6c80498a4b 100644 --- a/src/Dictionaries/DictionaryStructure.h +++ b/src/Dictionaries/DictionaryStructure.h @@ -158,6 +158,8 @@ struct DictionaryStructure final std::string getKeyDescription() const; bool isKeySizeFixed() const; size_t getKeySize() const; + Strings getKeysNames() const; + private: /// range_min and range_max have to be parsed before this function call std::vector getAttributes( diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index f2abe10f970..d9aa89ec14e 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -26,6 +26,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int DICTIONARY_ACCESS_DENIED; + extern const int UNSUPPORTED_METHOD; } namespace @@ -65,18 +66,32 @@ ExecutableDictionarySource::ExecutableDictionarySource( const Context & context_) : log(&Poco::Logger::get("ExecutableDictionarySource")) , dict_struct{dict_struct_} + , implicit_key{config.getBool(config_prefix + ".implicit_key", false)} , command{config.getString(config_prefix + ".command")} , update_field{config.getString(config_prefix + ".update_field", "")} , format{config.getString(config_prefix + ".format")} , sample_block{sample_block_} , context(context_) { + /// Remove keys from sample_block for implicit_key dictionary because + /// this columns will not be provided by client + if (implicit_key) + { + auto keys_names = dict_struct.getKeysNames(); + + for (auto & key_name : keys_names) + { + size_t key_column_position_in_block = sample_block.getPositionByName(key_name); + sample_block.erase(key_column_position_in_block); + } + } } ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionarySource & other) : log(&Poco::Logger::get("ExecutableDictionarySource")) , update_time{other.update_time} , dict_struct{other.dict_struct} + , implicit_key{other.implicit_key} , command{other.command} , update_field{other.update_field} , format{other.format} @@ -87,6 +102,9 @@ ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionar BlockInputStreamPtr ExecutableDictionarySource::loadAll() { + if (implicit_key) + throw Exception("ExecutableDictionarySource with implicit_key does not support loadAll method", ErrorCodes::UNSUPPORTED_METHOD); + LOG_TRACE(log, "loadAll {}", toString()); auto process = ShellCommand::execute(command); auto input_stream = context.getInputFormat(format, process->out, sample_block, max_block_size); @@ -95,6 +113,9 @@ BlockInputStreamPtr ExecutableDictionarySource::loadAll() BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll() { + if (implicit_key) + throw Exception("ExecutableDictionarySource with implicit_key does not support loadUpdatedAll method", ErrorCodes::UNSUPPORTED_METHOD); + time_t new_update_time = time(nullptr); SCOPE_EXIT(update_time = new_update_time); @@ -173,6 +194,72 @@ namespace std::function send_data; ThreadFromGlobalPool thread; }; + + /** A stream, adds additional columns to each block that it will read from inner stream. + * + * block_to_add rows size must be equal to final summ rows size of all inner stream readed blocks. + */ + class BlockInputStreamWithAdditionalColumns final: public IBlockInputStream + { + public: + BlockInputStreamWithAdditionalColumns( + Block block_to_add_, + std::unique_ptr&& stream_) + : block_to_add(std::move(block_to_add_)) + , stream(std::move(stream_)) + { + } + + Block getHeader() const override + { + auto header = stream->getHeader(); + + if (header) + { + for (int64_t i = static_cast(block_to_add.columns() - 1); i >= 0; --i) + header.insert(0, block_to_add.getByPosition(i).cloneEmpty()); + } + + return header; + } + + Block readImpl() override + { + auto block = stream->read(); + + if (block) + { + auto block_rows = block.rows(); + + auto cut_block = block_to_add.cloneWithCutColumns(current_range_index, block_rows); + + for (int64_t i = static_cast(cut_block.columns() - 1); i >= 0; --i) + block.insert(0, cut_block.getByPosition(i)); + + current_range_index += block_rows; + } + + return block; + } + + void readPrefix() override + { + stream->readPrefix(); + } + + void readSuffix() override + { + stream->readSuffix(); + } + + String getName() const override { return "BlockInputStreamWithAdditionalColumns"; } + + private: + Block block_to_add; + std::unique_ptr stream; + size_t current_range_index = 0; + }; + } @@ -180,28 +267,44 @@ BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector( + auto block = blockForIds(ids); + + auto stream = std::make_unique( context, format, sample_block, command, log, - [&ids, this](WriteBufferFromFile & out) mutable + [block, this](WriteBufferFromFile & out) mutable { - auto output_stream = context.getOutputStream(format, out, sample_block); - formatIDs(output_stream, ids); + auto output_stream = context.getOutputStream(format, out, block.cloneEmpty()); + formatWithBlock(output_stream, block); out.close(); }); + + if (implicit_key) + { + return std::make_shared(block, std::move(stream)); + } + else + return std::shared_ptr(stream.release()); } BlockInputStreamPtr ExecutableDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size()); - return std::make_shared( + auto block = blockForKeys(dict_struct, key_columns, requested_rows); + + auto stream = std::make_unique( context, format, sample_block, command, log, - [key_columns, &requested_rows, this](WriteBufferFromFile & out) mutable + [block, this](WriteBufferFromFile & out) mutable { - auto output_stream = context.getOutputStream(format, out, sample_block); - formatKeys(dict_struct, output_stream, key_columns, requested_rows); + auto output_stream = context.getOutputStream(format, out, block.cloneEmpty()); + formatWithBlock(output_stream, block); out.close(); }); + + if (implicit_key) + return std::make_shared(block, std::move(stream)); + else + return std::shared_ptr(stream.release()); } bool ExecutableDictionarySource::isModified() const diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index f28d71ca5e3..7aa203f267b 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -49,9 +49,9 @@ public: private: Poco::Logger * log; - time_t update_time = 0; const DictionaryStructure dict_struct; + bool implicit_key; const std::string command; const std::string update_field; const std::string format; diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index da5623bcdb0..55585b836d9 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -130,12 +130,14 @@ BlockInputStreamPtr HTTPDictionarySource::loadUpdatedAll() BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector & ids) { LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size()); + + auto block = blockForIds(ids); - ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [&](std::ostream & ostr) + ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [block, this](std::ostream & ostr) { WriteBufferFromOStream out_buffer(ostr); auto output_stream = context.getOutputStream(format, out_buffer, sample_block); - formatIDs(output_stream, ids); + formatWithBlock(output_stream, block); }; Poco::URI uri(url); @@ -150,11 +152,13 @@ BlockInputStreamPtr HTTPDictionarySource::loadKeys(const Columns & key_columns, { LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size()); - ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [&](std::ostream & ostr) + auto block = blockForKeys(dict_struct, key_columns, requested_rows); + + ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [block, this](std::ostream & ostr) { WriteBufferFromOStream out_buffer(ostr); auto output_stream = context.getOutputStream(format, out_buffer, sample_block); - formatKeys(dict_struct, output_stream, key_columns, requested_rows); + formatWithBlock(output_stream, block); }; Poco::URI uri(url); diff --git a/tests/config/executable_dictionary.xml b/tests/config/executable_dictionary.xml index 50df32e2ec6..c5a4a0947bc 100644 --- a/tests/config/executable_dictionary.xml +++ b/tests/config/executable_dictionary.xml @@ -105,4 +105,152 @@ + + simple_executable_cache_dictionary_no_implicit_key + + + + id + UInt64 + + + + value + String + + + + + + + echo "1\tValue" + TabSeparated + false + + + + + + 10000 + + + + 300 + + + + simple_executable_cache_dictionary_implicit_key + + + + id + UInt64 + + + + value + String + + + + + + + echo "Value" + TabSeparated + true + + + + + + 10000 + + + + 300 + + + + complex_executable_cache_dictionary_no_implicit_key + + + + + id + UInt64 + + + + id_key + String + + + + + + value + String + + + + + + + echo "1\tFirstKey\tValue" + TabSeparated + false + + + + + + 10000 + + + + 300 + + + + complex_executable_cache_dictionary_implicit_key + + + + + id + UInt64 + + + + id_key + String + + + + + + value + String + + + + + + + echo "Value" + TabSeparated + true + + + + + + 10000 + + + + 300 + + diff --git a/tests/queries/0_stateless/01474_executable_dictionary.reference b/tests/queries/0_stateless/01474_executable_dictionary.reference index 4d0994b08c3..66894d4f41f 100644 --- a/tests/queries/0_stateless/01474_executable_dictionary.reference +++ b/tests/queries/0_stateless/01474_executable_dictionary.reference @@ -1,3 +1,8 @@ 999999 1999998 999998000001 999999 1999998 999998000001 999999 1999998 999998000001 +Check implicit_key option +Value +Value +Value +Value \ No newline at end of file diff --git a/tests/queries/0_stateless/01474_executable_dictionary.sql b/tests/queries/0_stateless/01474_executable_dictionary.sql index 727cf47f79f..83f8a946354 100644 --- a/tests/queries/0_stateless/01474_executable_dictionary.sql +++ b/tests/queries/0_stateless/01474_executable_dictionary.sql @@ -1,3 +1,11 @@ SELECT number, dictGet('executable_complex', 'a', (number, number)) AS a, dictGet('executable_complex', 'b', (number, number)) AS b FROM numbers(1000000) WHERE number = 999999; SELECT number, dictGet('executable_complex_direct', 'a', (number, number)) AS a, dictGet('executable_complex_direct', 'b', (number, number)) AS b FROM numbers(1000000) WHERE number = 999999; SELECT number, dictGet('executable_simple', 'a', number) AS a, dictGet('executable_simple', 'b', number) AS b FROM numbers(1000000) WHERE number = 999999; + +SELECT 'Check implicit_key option'; + +SELECT dictGet('simple_executable_cache_dictionary_no_implicit_key', 'value', toUInt64(1)); +SELECT dictGet('simple_executable_cache_dictionary_implicit_key', 'value', toUInt64(1)); + +SELECT dictGet('complex_executable_cache_dictionary_no_implicit_key', 'value', (toUInt64(1), 'FirstKey')); +SELECT dictGet('complex_executable_cache_dictionary_implicit_key', 'value', (toUInt64(1), 'FirstKey')); From d194294d0b741b342011e73575d6483ffde5e467 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 27 Jan 2021 16:23:02 +0300 Subject: [PATCH 66/94] Fixed tests --- src/Dictionaries/DictionarySourceHelpers.cpp | 10 +++++----- src/Dictionaries/DictionarySourceHelpers.h | 6 ++++-- src/Dictionaries/ExecutableDictionarySource.cpp | 2 +- src/Dictionaries/HTTPDictionarySource.cpp | 4 ++-- .../0_stateless/01474_executable_dictionary.reference | 2 +- 5 files changed, 13 insertions(+), 11 deletions(-) diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index 3227704ff4b..ff769d51354 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -13,7 +13,7 @@ namespace DB { -void formatWithBlock(BlockOutputStreamPtr & out, Block block) +void formatWithBlock(BlockOutputStreamPtr & out, const Block & block) { out->writePrefix(); out->write(block); @@ -23,14 +23,14 @@ void formatWithBlock(BlockOutputStreamPtr & out, Block block) /// For simple key -Block blockForIds(const std::vector & ids) +Block blockForIds( + const DictionaryStructure & dict_struct, + const std::vector & ids) { auto column = ColumnUInt64::create(ids.size()); memcpy(column->getData().data(), ids.data(), ids.size() * sizeof(ids.front())); - Block block{{std::move(column), std::make_shared(), "id"}}; - - std::cerr << "Block for IDs size " << ids.size() << std::endl; + Block block{{std::move(column), std::make_shared(), (*dict_struct.id).name}}; return block; } diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index d6b0dab8d4f..27ead1b447d 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -20,11 +20,13 @@ class Context; /// Write keys to block output stream. -void formatWithBlock(BlockOutputStreamPtr & out, Block block); +void formatWithBlock(BlockOutputStreamPtr & out, const Block & block); /// For simple key -Block blockForIds(const std::vector & ids); +Block blockForIds( + const DictionaryStructure & dict_struct, + const std::vector & ids); /// For composite key diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index d9aa89ec14e..19eb6a82672 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -267,7 +267,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector( context, format, sample_block, command, log, diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 55585b836d9..75daf1323e8 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -130,8 +130,8 @@ BlockInputStreamPtr HTTPDictionarySource::loadUpdatedAll() BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector & ids) { LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size()); - - auto block = blockForIds(ids); + + auto block = blockForIds(dict_struct, ids); ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = [block, this](std::ostream & ostr) { diff --git a/tests/queries/0_stateless/01474_executable_dictionary.reference b/tests/queries/0_stateless/01474_executable_dictionary.reference index 66894d4f41f..d88c4d078e3 100644 --- a/tests/queries/0_stateless/01474_executable_dictionary.reference +++ b/tests/queries/0_stateless/01474_executable_dictionary.reference @@ -5,4 +5,4 @@ Check implicit_key option Value Value Value -Value \ No newline at end of file +Value From a9ecb6bf78cc72d6c744094df37c2c3fa8cf63fc Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 28 Jan 2021 20:10:27 +0300 Subject: [PATCH 67/94] Fixed clang-tidy --- src/Core/Block.cpp | 4 ++-- src/Dictionaries/DictionaryStructure.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Core/Block.cpp b/src/Core/Block.cpp index 56c045bc8df..0c9a470dc1d 100644 --- a/src/Core/Block.cpp +++ b/src/Core/Block.cpp @@ -413,8 +413,8 @@ Block Block::cloneWithCutColumns(size_t start, size_t length) const { Block copy = *this; - for (size_t i = 0; i < copy.data.size(); ++i) - copy.data[i].column = copy.data[i].column->cut(start, length); + for (auto & column_to_cut : copy.data) + column_to_cut.column = column_to_cut.column->cut(start, length); return copy; } diff --git a/src/Dictionaries/DictionaryStructure.cpp b/src/Dictionaries/DictionaryStructure.cpp index 42a7cb6e4ec..25f5736221e 100644 --- a/src/Dictionaries/DictionaryStructure.cpp +++ b/src/Dictionaries/DictionaryStructure.cpp @@ -286,7 +286,7 @@ Strings DictionaryStructure::getKeysNames() const if (id) return { id->name }; - auto & key_attributes = *key; + const auto & key_attributes = *key; Strings keys_names; keys_names.reserve(key_attributes.size()); From a0755e439c0b20eff24044462b2787590a5c731a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 28 Jan 2021 22:57:11 +0300 Subject: [PATCH 68/94] Fixed typo --- src/Dictionaries/ExecutableDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 19eb6a82672..e6bf18d3239 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -197,7 +197,7 @@ namespace /** A stream, adds additional columns to each block that it will read from inner stream. * - * block_to_add rows size must be equal to final summ rows size of all inner stream readed blocks. + * block_to_add rows size must be equal to final summ rows size of all inner stream blocks. */ class BlockInputStreamWithAdditionalColumns final: public IBlockInputStream { From b83b8c56dac5045ef93f4cc47e924379dc881ebf Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 31 Jan 2021 00:06:16 +0300 Subject: [PATCH 69/94] Update ExecutableDictionarySource.cpp --- src/Dictionaries/ExecutableDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index e6bf18d3239..40addc574b9 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -197,7 +197,7 @@ namespace /** A stream, adds additional columns to each block that it will read from inner stream. * - * block_to_add rows size must be equal to final summ rows size of all inner stream blocks. + * block_to_add rows size must be equal to final sum rows size of all inner stream blocks. */ class BlockInputStreamWithAdditionalColumns final: public IBlockInputStream { From ecd6e10e2208e8b6ecfcddd4b94d4c4f1dc1170d Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 31 Jan 2021 01:27:04 +0300 Subject: [PATCH 70/94] Update ExecutableDictionarySource.cpp --- src/Dictionaries/ExecutableDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 40addc574b9..be0d3a4afa7 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -74,7 +74,7 @@ ExecutableDictionarySource::ExecutableDictionarySource( , context(context_) { /// Remove keys from sample_block for implicit_key dictionary because - /// this columns will not be provided by client + /// these columns will not be provided by client if (implicit_key) { auto keys_names = dict_struct.getKeysNames(); From 77a8ef24de2dadd9f0a09f9c48d6b61f914a69f7 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 31 Jan 2021 01:29:36 +0300 Subject: [PATCH 71/94] Update ExecutableDictionarySource.cpp --- src/Dictionaries/ExecutableDictionarySource.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index be0d3a4afa7..48cd2b130d4 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -74,7 +74,9 @@ ExecutableDictionarySource::ExecutableDictionarySource( , context(context_) { /// Remove keys from sample_block for implicit_key dictionary because - /// these columns will not be provided by client + /// these columns will not be returned from source + /// Implicit key means that the source script will return only values, + /// and the correspondence to keys is determined implicitly - by the order of rows in the result. if (implicit_key) { auto keys_names = dict_struct.getKeysNames(); From e5b89f24b57fcd52b32b34b5ec60e0e485141233 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 31 Jan 2021 01:30:00 +0300 Subject: [PATCH 72/94] Update ExecutableDictionarySource.cpp --- src/Dictionaries/ExecutableDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 48cd2b130d4..fd6991d3e97 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -76,7 +76,7 @@ ExecutableDictionarySource::ExecutableDictionarySource( /// Remove keys from sample_block for implicit_key dictionary because /// these columns will not be returned from source /// Implicit key means that the source script will return only values, - /// and the correspondence to keys is determined implicitly - by the order of rows in the result. + /// and the correspondence to the requested keys is determined implicitly - by the order of rows in the result. if (implicit_key) { auto keys_names = dict_struct.getKeysNames(); From d3f299269f64fd7a636303481e964b2688ae4a7f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 31 Jan 2021 12:59:35 +0300 Subject: [PATCH 73/94] Fixed code review issues --- src/Dictionaries/DictionarySourceHelpers.cpp | 2 +- src/Dictionaries/DictionarySourceHelpers.h | 2 +- src/Dictionaries/ExecutableDictionarySource.cpp | 12 ++++++++---- src/Dictionaries/HTTPDictionarySource.cpp | 4 ++-- .../01474_executable_dictionary.reference | 5 ----- .../0_stateless/01474_executable_dictionary.sql | 8 -------- ...1674_executable_dictionary_implicit_key.reference | 4 ++++ .../01674_executable_dictionary_implicit_key.sql | 5 +++++ 8 files changed, 21 insertions(+), 21 deletions(-) create mode 100644 tests/queries/0_stateless/01674_executable_dictionary_implicit_key.reference create mode 100644 tests/queries/0_stateless/01674_executable_dictionary_implicit_key.sql diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index ff769d51354..2a872672aff 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -13,7 +13,7 @@ namespace DB { -void formatWithBlock(BlockOutputStreamPtr & out, const Block & block) +void formatBlock(BlockOutputStreamPtr & out, const Block & block) { out->writePrefix(); out->write(block); diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index 27ead1b447d..cad5441c66e 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -20,7 +20,7 @@ class Context; /// Write keys to block output stream. -void formatWithBlock(BlockOutputStreamPtr & out, const Block & block); +void formatBlock(BlockOutputStreamPtr & out, const Block & block); /// For simple key diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index fd6991d3e97..fc061f93f2b 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -218,7 +218,7 @@ namespace if (header) { - for (int64_t i = static_cast(block_to_add.columns() - 1); i >= 0; --i) + for (Int64 i = static_cast(block_to_add.columns() - 1); i >= 0; --i) header.insert(0, block_to_add.getByPosition(i).cloneEmpty()); } @@ -235,7 +235,11 @@ namespace auto cut_block = block_to_add.cloneWithCutColumns(current_range_index, block_rows); - for (int64_t i = static_cast(cut_block.columns() - 1); i >= 0; --i) + if (cut_block.rows() != block_rows) + throw Exception("Rows in block to add after cut must equal to rows in readed block", + ErrorCodes::LOGICAL_ERROR); + + for (Int64 i = static_cast(cut_block.columns() - 1); i >= 0; --i) block.insert(0, cut_block.getByPosition(i)); current_range_index += block_rows; @@ -276,7 +280,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector & id { WriteBufferFromOStream out_buffer(ostr); auto output_stream = context.getOutputStream(format, out_buffer, sample_block); - formatWithBlock(output_stream, block); + formatBlock(output_stream, block); }; Poco::URI uri(url); @@ -158,7 +158,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadKeys(const Columns & key_columns, { WriteBufferFromOStream out_buffer(ostr); auto output_stream = context.getOutputStream(format, out_buffer, sample_block); - formatWithBlock(output_stream, block); + formatBlock(output_stream, block); }; Poco::URI uri(url); diff --git a/tests/queries/0_stateless/01474_executable_dictionary.reference b/tests/queries/0_stateless/01474_executable_dictionary.reference index d88c4d078e3..4d0994b08c3 100644 --- a/tests/queries/0_stateless/01474_executable_dictionary.reference +++ b/tests/queries/0_stateless/01474_executable_dictionary.reference @@ -1,8 +1,3 @@ 999999 1999998 999998000001 999999 1999998 999998000001 999999 1999998 999998000001 -Check implicit_key option -Value -Value -Value -Value diff --git a/tests/queries/0_stateless/01474_executable_dictionary.sql b/tests/queries/0_stateless/01474_executable_dictionary.sql index 83f8a946354..727cf47f79f 100644 --- a/tests/queries/0_stateless/01474_executable_dictionary.sql +++ b/tests/queries/0_stateless/01474_executable_dictionary.sql @@ -1,11 +1,3 @@ SELECT number, dictGet('executable_complex', 'a', (number, number)) AS a, dictGet('executable_complex', 'b', (number, number)) AS b FROM numbers(1000000) WHERE number = 999999; SELECT number, dictGet('executable_complex_direct', 'a', (number, number)) AS a, dictGet('executable_complex_direct', 'b', (number, number)) AS b FROM numbers(1000000) WHERE number = 999999; SELECT number, dictGet('executable_simple', 'a', number) AS a, dictGet('executable_simple', 'b', number) AS b FROM numbers(1000000) WHERE number = 999999; - -SELECT 'Check implicit_key option'; - -SELECT dictGet('simple_executable_cache_dictionary_no_implicit_key', 'value', toUInt64(1)); -SELECT dictGet('simple_executable_cache_dictionary_implicit_key', 'value', toUInt64(1)); - -SELECT dictGet('complex_executable_cache_dictionary_no_implicit_key', 'value', (toUInt64(1), 'FirstKey')); -SELECT dictGet('complex_executable_cache_dictionary_implicit_key', 'value', (toUInt64(1), 'FirstKey')); diff --git a/tests/queries/0_stateless/01674_executable_dictionary_implicit_key.reference b/tests/queries/0_stateless/01674_executable_dictionary_implicit_key.reference new file mode 100644 index 00000000000..0777c3c2625 --- /dev/null +++ b/tests/queries/0_stateless/01674_executable_dictionary_implicit_key.reference @@ -0,0 +1,4 @@ +Value +Value +Value +Value diff --git a/tests/queries/0_stateless/01674_executable_dictionary_implicit_key.sql b/tests/queries/0_stateless/01674_executable_dictionary_implicit_key.sql new file mode 100644 index 00000000000..c98cb0a5e0f --- /dev/null +++ b/tests/queries/0_stateless/01674_executable_dictionary_implicit_key.sql @@ -0,0 +1,5 @@ +SELECT dictGet('simple_executable_cache_dictionary_no_implicit_key', 'value', toUInt64(1)); +SELECT dictGet('simple_executable_cache_dictionary_implicit_key', 'value', toUInt64(1)); + +SELECT dictGet('complex_executable_cache_dictionary_no_implicit_key', 'value', (toUInt64(1), 'FirstKey')); +SELECT dictGet('complex_executable_cache_dictionary_implicit_key', 'value', (toUInt64(1), 'FirstKey')); From d9d206af92918394429a775decf8447c29c0cf29 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 31 Jan 2021 15:26:53 +0300 Subject: [PATCH 74/94] Fixed typo --- src/Dictionaries/ExecutableDictionarySource.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index fc061f93f2b..42dac540f09 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -27,6 +27,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int DICTIONARY_ACCESS_DENIED; extern const int UNSUPPORTED_METHOD; + extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; } namespace @@ -236,8 +237,9 @@ namespace auto cut_block = block_to_add.cloneWithCutColumns(current_range_index, block_rows); if (cut_block.rows() != block_rows) - throw Exception("Rows in block to add after cut must equal to rows in readed block", - ErrorCodes::LOGICAL_ERROR); + throw Exception( + "Number of rows in block to add after cut must equal to number of rows in block from inner stream", + ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); for (Int64 i = static_cast(cut_block.columns() - 1); i >= 0; --i) block.insert(0, cut_block.getByPosition(i)); From beee7a4f025bf4a75367e5607d65b34cd28fe952 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 1 Feb 2021 11:19:08 +0300 Subject: [PATCH 75/94] Added test to arcadia_skip_list --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 253eab720af..5b8256bb5af 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -198,3 +198,4 @@ 01659_test_base64Decode_mysql_compatibility 01675_data_type_coroutine 01671_aggregate_function_group_bitmap_data +01674_executable_dictionary_implicit_key From c9b78186d91db9c76258e6eb5a3c54af430bce43 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 1 Feb 2021 12:48:27 +0300 Subject: [PATCH 76/94] Auto version update to [21.2.1.5869] [54447] --- cmake/autogenerated_versions.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 5d643cc4bee..a13725a2c08 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -3,7 +3,7 @@ SET(VERSION_REVISION 54447) SET(VERSION_MAJOR 21) SET(VERSION_MINOR 2) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH 53d0c9fa7255aa1dc48991d19f4246ff71cc2fd7) -SET(VERSION_DESCRIBE v21.2.1.1-prestable) -SET(VERSION_STRING 21.2.1.1) +SET(VERSION_GITHASH ef72ba7349f230321750c13ee63b49a11a7c0adc) +SET(VERSION_DESCRIBE v21.2.1.5869-prestable) +SET(VERSION_STRING 21.2.1.5869) # end of autochange From cd82eed093513675da19a303f38884caee07aa47 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 1 Feb 2021 12:50:56 +0300 Subject: [PATCH 77/94] Auto version update to [21.3.1.1] [54448] --- cmake/autogenerated_versions.txt | 8 ++++---- debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- .../StorageSystemContributors.generated.cpp | 19 +++++++++++++++++++ 6 files changed, 28 insertions(+), 9 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index a13725a2c08..ce92ae203ea 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -1,9 +1,9 @@ # This strings autochanged from release_lib.sh: -SET(VERSION_REVISION 54447) +SET(VERSION_REVISION 54448) SET(VERSION_MAJOR 21) -SET(VERSION_MINOR 2) +SET(VERSION_MINOR 3) SET(VERSION_PATCH 1) SET(VERSION_GITHASH ef72ba7349f230321750c13ee63b49a11a7c0adc) -SET(VERSION_DESCRIBE v21.2.1.5869-prestable) -SET(VERSION_STRING 21.2.1.5869) +SET(VERSION_DESCRIBE v21.3.1.1-prestable) +SET(VERSION_STRING 21.3.1.1) # end of autochange diff --git a/debian/changelog b/debian/changelog index 1cec020f026..53b36cae114 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (21.2.1.1) unstable; urgency=low +clickhouse (21.3.1.1) unstable; urgency=low * Modified source code - -- clickhouse-release Mon, 11 Jan 2021 11:12:08 +0300 + -- clickhouse-release Mon, 01 Feb 2021 12:50:53 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index 5022687c47b..43921a4d3c4 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.2.1.* +ARG version=21.3.1.* RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 3528ae68ef6..8e39af5646c 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:20.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.2.1.* +ARG version=21.3.1.* ARG gosu_ver=1.10 # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index df918928f99..f151ae8fddf 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/" -ARG version=21.2.1.* +ARG version=21.3.1.* RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index 0c50e452e95..d9f4a2a7dd0 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -23,6 +23,7 @@ const char * auto_contributors[] { "Alexander Burmak", "Alexander Ermolaev", "Alexander GQ Gerasiov", + "Alexander Gololobov", "Alexander Kazakov", "Alexander Kozhikhov", "Alexander Krasheninnikov", @@ -43,6 +44,7 @@ const char * auto_contributors[] { "Alexandr Krasheninnikov", "Alexandr Orlov", "Alexandra Latysheva", + "Alexandre Snarskii", "Alexei Averchenko", "Alexey", "Alexey Arno", @@ -143,6 +145,7 @@ const char * auto_contributors[] { "CurtizJ", "Daniel Bershatsky", "Daniel Dao", + "Daniel Qin", "Danila Kutenin", "Dao Minh Thuc", "Daria Mozhaeva", @@ -309,7 +312,9 @@ const char * auto_contributors[] { "Marek Vavrusa", "Marek Vavruša", "Marek Vavruša", + "Mariano Benítez Mulet", "Mark Andreev", + "Mark Frost", "Mark Papadakis", "Maroun Maroun", "Marsel Arduanov", @@ -422,6 +427,7 @@ const char * auto_contributors[] { "Rafael David Tinoco", "Ramazan Polat", "Ravengg", + "RegulusZ", "Reilee", "Reto Kromer", "Ri", @@ -482,9 +488,11 @@ const char * auto_contributors[] { "Tangaev", "Tema Novikov", "The-Alchemist", + "TiunovNN", "Tobias Adamson", "Tom Bombadil", "Tsarkova Anastasia", + "TszkitLo40", "Ubuntu", "Ubus", "UnamedRus", @@ -556,6 +564,7 @@ const char * auto_contributors[] { "Yury Stankevich", "Zhichang Yu", "Zhipeng", + "Zoran Pandovski", "a.palagashvili", "abdrakhmanov", "abyss7", @@ -571,6 +580,7 @@ const char * auto_contributors[] { "alex.lvxin", "alexander kozhikhov", "alexey-milovidov", + "alfredlu", "amoschen", "amudong", "ana-uvarova", @@ -588,14 +598,17 @@ const char * auto_contributors[] { "avsharapov", "awesomeleo", "benamazing", + "benbiti", "bgranvea", "bharatnc", "blazerer", "bluebirddm", + "bo zeng", "bobrovskij artemij", "booknouse", "bseng", "cekc", + "centos7", "champtar", "chang.chen", "chengy8934", @@ -606,6 +619,7 @@ const char * auto_contributors[] { "comunodi", "coraxster", "damozhaeva", + "dankondr", "daoready", "dasmfm", "davydovska", @@ -627,6 +641,7 @@ const char * auto_contributors[] { "elBroom", "elenaspb2019", "emakarov", + "emhlbmc", "emironyuk", "evtan", "exprmntr", @@ -673,6 +688,7 @@ const char * auto_contributors[] { "javi santana", "jetgm", "jianmei zhang", + "jyz0309", "kmeaw", "koshachy", "kreuzerkrieg", @@ -779,7 +795,9 @@ const char * auto_contributors[] { "taiyang-li", "tao jiang", "tavplubix", + "templarzq", "tiger.yan", + "tison", "topvisor", "tyrionhuang", "ubuntu", @@ -800,6 +818,7 @@ const char * auto_contributors[] { "weeds085490", "xPoSx", "yangshuai", + "ygrek", "yhgcn", "ylchou", "yonesko", From 1e44e3f3417b3d23d8ef3e840c6a11f0716645f0 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 1 Feb 2021 12:53:13 +0300 Subject: [PATCH 78/94] Update 01671_ddl_hang_timeout.sh --- tests/queries/0_stateless/01671_ddl_hang_timeout.sh | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01671_ddl_hang_timeout.sh b/tests/queries/0_stateless/01671_ddl_hang_timeout.sh index fb4926c8b00..2ca97e3978b 100755 --- a/tests/queries/0_stateless/01671_ddl_hang_timeout.sh +++ b/tests/queries/0_stateless/01671_ddl_hang_timeout.sh @@ -2,22 +2,23 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS test_01671" -$CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01671" + function thread_create_drop_table { while true; do REPLICA=$(($RANDOM % 10)) - $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS test_01671.t1 (x UInt64, s Array(Nullable(String))) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01671/test_01671', 'r_$REPLICA') order by x" 2>/dev/null + $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS t1 (x UInt64, s Array(Nullable(String))) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01671/test_01671', 'r_$REPLICA') order by x" 2>/dev/null sleep 0.0$RANDOM - $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_01671.t1" + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t1" done } + function thread_alter_table { while true; do - $CLICKHOUSE_CLIENT --query "ALTER TABLE test_01671.t1 on cluster test_shard_localhost ADD COLUMN newcol UInt32" >/dev/null 2>&1 + $CLICKHOUSE_CLIENT --query "ALTER TABLE $CLICKHOUSE_DATABASE.t1 on cluster test_shard_localhost ADD COLUMN newcol UInt32" >/dev/null 2>&1 sleep 0.0$RANDOM done } + export -f thread_create_drop_table export -f thread_alter_table timeout 20 bash -c "thread_create_drop_table" & @@ -25,4 +26,4 @@ timeout 20 bash -c 'thread_alter_table' & wait sleep 1 -$CLICKHOUSE_CLIENT --query "DROP DATABASE test_01671"; +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t1"; From 7780c0b8e933b8d00bbe832ccafc2848130c6ede Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Mon, 1 Feb 2021 13:49:19 +0300 Subject: [PATCH 79/94] Remove an always-false condition from query parser https://github.com/ClickHouse/ClickHouse/pull/19236#discussion_r567673775 --- src/Parsers/parseQuery.cpp | 8 -------- 1 file changed, 8 deletions(-) diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index 48a92534e74..46f9fb3310d 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -269,14 +269,6 @@ ASTPtr tryParseQuery( // most of the checks. if (insert && insert->data) { - if (!parse_res) - { - // Generic parse error. - out_error_message = getSyntaxErrorMessage(query_begin, all_queries_end, - last_token, expected, hilite, query_description); - return nullptr; - } - return res; } From accf4d262e904aca42d8f8ffdb2f864e3cfe2b94 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 1 Feb 2021 15:40:57 +0300 Subject: [PATCH 80/94] fix logical error --- .../Formats/Impl/ValuesBlockInputFormat.cpp | 10 +++++ .../01179_insert_values_semicolon.expect | 39 +++++++++++++++++++ .../01179_insert_values_semicolon.reference | 0 3 files changed, 49 insertions(+) create mode 100755 tests/queries/0_stateless/01179_insert_values_semicolon.expect create mode 100644 tests/queries/0_stateless/01179_insert_values_semicolon.reference diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index c3b753e7261..34a4a98f16b 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -24,6 +24,7 @@ namespace ErrorCodes extern const int TYPE_MISMATCH; extern const int SUPPORT_IS_DISABLED; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int CANNOT_READ_ALL_DATA; } @@ -412,6 +413,15 @@ void ValuesBlockInputFormat::readPrefix() void ValuesBlockInputFormat::readSuffix() { + if (!buf.eof() && *buf.position() == ';') + { + ++buf.position(); + skipWhitespaceIfAny(buf); + if (buf.hasUnreadData()) + throw Exception("Cannot read data after semicolon", ErrorCodes::CANNOT_READ_ALL_DATA); + return; + } + if (buf.hasUnreadData()) throw Exception("Unread data in PeekableReadBuffer will be lost. Most likely it's a bug.", ErrorCodes::LOGICAL_ERROR); } diff --git a/tests/queries/0_stateless/01179_insert_values_semicolon.expect b/tests/queries/0_stateless/01179_insert_values_semicolon.expect new file mode 100755 index 00000000000..c832be72c10 --- /dev/null +++ b/tests/queries/0_stateless/01179_insert_values_semicolon.expect @@ -0,0 +1,39 @@ +#!/usr/bin/expect -f + +log_user 0 +set timeout 5 +match_max 100000 +# A default timeout action is to do nothing, change it to fail +expect_after { + timeout { + exit 1 + } +} + +set basedir [file dirname $argv0] +spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT" +expect ":) " + +send -- "DROP TABLE IF EXISTS test_01179\r" +expect "Ok." + +send -- "CREATE TABLE test_01179 (date DateTime) ENGINE=Memory()\r" +expect "Ok." + +send -- "INSERT INTO test_01179 values ('2020-01-01')\r" +expect "Ok." + +send -- "INSERT INTO test_01179 values ('2020-01-01'); \r" +expect "Ok." + +send -- "INSERT INTO test_01179 values ('2020-01-01'); (1) \r" +expect "Cannot read data after semicolon" + +send -- "SELECT date, count() FROM test_01179 GROUP BY date FORMAT TSV\r" +expect "2020-01-01 00:00:00\t3" + +send -- "DROP TABLE test_01179\r" +expect "Ok." + +send -- "\4" +expect eof diff --git a/tests/queries/0_stateless/01179_insert_values_semicolon.reference b/tests/queries/0_stateless/01179_insert_values_semicolon.reference new file mode 100644 index 00000000000..e69de29bb2d From c4a3acd4f82d441e8f58367149167543def26b3d Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 1 Feb 2021 17:53:38 +0300 Subject: [PATCH 81/94] 2021 footer --- website/templates/footer.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/templates/footer.html b/website/templates/footer.html index 765ea63d528..1eaf519b58b 100644 --- a/website/templates/footer.html +++ b/website/templates/footer.html @@ -8,7 +8,7 @@ {{ _('ClickHouse source code is published under the Apache 2.0 License.') }} {{ _('Software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.') }}
- © 2016–2020 {{ _('Yandex LLC') }} + © 2016–2021 {{ _('Yandex LLC') }}
From e6fd6abe6f78ee017236b8fa059635ad0fbfcb37 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 1 Feb 2021 18:07:34 +0300 Subject: [PATCH 82/94] Fix test keeper integration tests --- tests/integration/test_testkeeper_back_to_back/test.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_testkeeper_back_to_back/test.py b/tests/integration/test_testkeeper_back_to_back/test.py index f74d6a4c646..d3a9b742cdd 100644 --- a/tests/integration/test_testkeeper_back_to_back/test.py +++ b/tests/integration/test_testkeeper_back_to_back/test.py @@ -8,7 +8,7 @@ from multiprocessing.dummy import Pool cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', main_configs=['configs/enable_test_keeper.xml', 'configs/logs_conf.xml'], with_zookeeper=True) -from kazoo.client import KazooClient +from kazoo.client import KazooClient, KazooState _genuine_zk_instance = None _fake_zk_instance = None @@ -26,6 +26,13 @@ def get_fake_zk(): if not _fake_zk_instance: print("node", cluster.get_instance_ip("node")) _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip("node") + ":9181") + def reset_last_zxid_listener(state): + print("Fake zk callback called for state", state) + global _fake_zk_instance + # reset last_zxid -- fake server doesn't support it + _fake_zk_instance.last_zxid = 0 + + _fake_zk_instance.add_listener(reset_last_zxid_listener) _fake_zk_instance.start() return _fake_zk_instance From a3f1b825cc8d419e8d082237d8331744f2dccbab Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 1 Feb 2021 21:17:12 +0300 Subject: [PATCH 83/94] Fix build --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 9a32b29454d..457c9c04aa9 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1549,7 +1549,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( if (!out_projection) out_projection = createProjection(pipe.getHeader()); - QueryPlanPtr plan = createPlanFromPipe(std::move(pipe), "with final"); + QueryPlanPtr plan = createPlanFromPipe(std::move(pipe), query_id, data, "with final"); auto expression_step = std::make_unique( plan->getCurrentDataStream(), From fb98aeeae1274f746c67d504fc0ec2933e16f4c8 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Tue, 2 Feb 2021 02:44:40 +0300 Subject: [PATCH 84/94] Fixes --- .../table-engines/mergetree-family/mergetree.md | 13 ++++++------- .../table-engines/mergetree-family/mergetree.md | 13 ++++++------- 2 files changed, 12 insertions(+), 14 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 75fc42b6fc6..202c476bba8 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -83,7 +83,7 @@ For a description of parameters, see the [CREATE query description](../../../sql Expression must have one `Date` or `DateTime` column as a result. Example: `TTL date + INTERVAL 1 DAY` - Type of the rule `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'` specifies an action to be done with the part if the expression is satisfied (reaches current time): removal of expired rows, moving a part (if expression is satisfied for all rows in a part) to specified disk (`TO DISK 'xxx'`) or to volume (`TO VOLUME 'xxx'`). Default type of the rule is removal (`DELETE`). List of multiple rules can specified, but there should be no more than one `DELETE` rule. + Type of the rule `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'|GROUP BY` specifies an action to be done with the part if the expression is satisfied (reaches current time): removal of expired rows, moving a part (if expression is satisfied for all rows in a part) to specified disk (`TO DISK 'xxx'`) or to volume (`TO VOLUME 'xxx'`), or aggregating values in expired rows. Default type of the rule is removal (`DELETE`). List of multiple rules can specified, but there should be no more than one `DELETE` rule. For more details, see [TTL for columns and tables](#table_engine-mergetree-ttl) @@ -468,15 +468,14 @@ Type of TTL rule may follow each TTL expression. It affects an action which is t - `DELETE` - delete expired rows (default action); - `TO DISK 'aaa'` - move part to the disk `aaa`; -- `TO VOLUME 'bbb'` - move part to the disk `bbb`. +- `TO VOLUME 'bbb'` - move part to the disk `bbb`; +- `GROUP BY` - aggregate expired rows. -With `WHERE` clause you may specify which of the expired rows to delete or move. +With `WHERE` clause you may specify which of the expired rows to delete or aggregate (it cannot be applied to moves). -With `GROUP BY` clause you may [aggregate](../../../sql-reference/aggregate-functions/index.md) expired rows. `GROUP BY` key expression must be a prefix of the table primary key. +`GROUP BY` expression must be a prefix of the table primary key. -If a column is part of primary key, but not present in `GROUP BY` key expression, in result rows it contains aggregated value across grouped rows. - -If a column is not present neither in primary key, nor in `SET` clause, in result row it contains any occasional value from grouped rows. +If a column is not part of the `GROUP BY` expression and is not set explicitely in the `SET` clause, in result row it contains an occasional value from the grouped rows (as if aggregate function `any` is applied to it). **Examples** diff --git a/docs/ru/engines/table-engines/mergetree-family/mergetree.md b/docs/ru/engines/table-engines/mergetree-family/mergetree.md index e21d4bc47e2..72a354bce22 100644 --- a/docs/ru/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/ru/engines/table-engines/mergetree-family/mergetree.md @@ -74,7 +74,7 @@ ORDER BY expr Выражение должно возвращать столбец `Date` или `DateTime`. Пример: `TTL date + INTERVAL 1 DAY`. - Тип правила `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'` указывает действие, которое будет выполнено с частью, удаление строк (прореживание), перемещение (при выполнении условия для всех строк части) на определённый диск (`TO DISK 'xxx'`) или том (`TO VOLUME 'xxx'`). Поведение по умолчанию соответствует удалению строк (`DELETE`). В списке правил может быть указано только одно выражение с поведением `DELETE`. + Тип правила `DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'|GROUP BY` указывает действие, которое будет выполнено с частью: удаление строк (прореживание), перемещение (при выполнении условия для всех строк части) на определённый диск (`TO DISK 'xxx'`) или том (`TO VOLUME 'xxx'`), или агрегирование данных в устаревших строках. Поведение по умолчанию соответствует удалению строк (`DELETE`). В списке правил может быть указано только одно выражение с поведением `DELETE`. Дополнительные сведения смотрите в разделе [TTL для столбцов и таблиц](#table_engine-mergetree-ttl) @@ -456,15 +456,14 @@ TTL expr - `DELETE` - удалить данные (действие по умолчанию); - `TO DISK 'aaa'` - переместить данные на диск `aaa`; -- `TO VOLUME 'bbb'` - переместить данные на том `bbb`. +- `TO VOLUME 'bbb'` - переместить данные на том `bbb`; +- `GROUP BY` - агрегировать данные. -В секции `WHERE` можно задать условие удаления или перемещения устаревших строк. +В секции `WHERE` можно задать условие удаления или агрегирования устаревших строк (для перемещения условие `WHERE` не применимо). -В секции `GROUP BY` можно [агрегировать](../../../sql-reference/aggregate-functions/index.md) данные из устаревших строк. Колонки, по которым агрегируются данные в `GROUP BY`, должны являться префиксом первичного ключа таблицы. +Колонки, по которым агрегируются данные в `GROUP BY`, должны являться префиксом первичного ключа таблицы. -Если колонка является частью первичного ключа, но не фигурирует в списке полей в `GROUP BY`, в результирующих строках она будет содержать агрегированные данные по сгруппированным строкам. - -Если колонка не является частью первичного ключа и не указана в секции `SET`, в результирующих строках она будет содержать случайное значение, взятое из одной из сгруппированных строк. +Если колонка не является частью выражения `GROUP BY` и не задается напрямую в секции `SET`, в результирующих строках она будет содержать случайное значение, взятое из одной из сгруппированных строк (как будто к ней применяется агрегирующая функция `any`). **Примеры** From b9fff3676b460d97f786e42b25d2c9be6d6420b5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 2 Feb 2021 06:03:00 +0300 Subject: [PATCH 85/94] Add exception for ANTLR tests --- tests/queries/skip_list.json | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index dbde52f040a..26c1c71de62 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -311,7 +311,8 @@ "01643_system_suspend", "01655_plan_optimizations", "01475_read_subcolumns_storages", - "01674_clickhouse_client_query_param_cte" + "01674_clickhouse_client_query_param_cte", + "01666_merge_tree_max_query_limit" ], "parallel": [ From bd0ec1b9f4ff7247c2176833f154e5486c589ba2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 2 Feb 2021 06:03:30 +0300 Subject: [PATCH 86/94] Remove useless header --- src/Formats/ProtobufWriter.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Formats/ProtobufWriter.h b/src/Formats/ProtobufWriter.h index 44d3aac221e..52bb453aa73 100644 --- a/src/Formats/ProtobufWriter.h +++ b/src/Formats/ProtobufWriter.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include From 9f8f9087797afb4fa962d6fc2e053dd441f23a47 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 2 Feb 2021 06:05:40 +0300 Subject: [PATCH 87/94] Better code (add suggestion from Azat) --- src/Interpreters/SystemLog.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 101bc752f43..84a2c075355 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -246,7 +246,7 @@ void SystemLog::add(const LogElement & element) /// The size of allocation can be in order of a few megabytes. /// But this should not be accounted for query memory usage. /// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flacky. - MemoryTracker::BlockerInThread temporarily_disable_memory_tracker; + MemoryTracker::BlockerInThread temporarily_disable_memory_tracker(VariableContext::Global); /// Should not log messages under mutex. bool queue_is_half_full = false; From 9930bb0bf6159082c8cc5d304552dd3c1bd7d9b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 2 Feb 2021 06:37:24 +0300 Subject: [PATCH 88/94] Fix UBSan report in geoHashesInBox --- src/Functions/GeoHash.cpp | 22 ++++++++++++------- .../0_stateless/01684_geohash_ubsan.reference | 1 + .../0_stateless/01684_geohash_ubsan.sql | 1 + 3 files changed, 16 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/01684_geohash_ubsan.reference create mode 100644 tests/queries/0_stateless/01684_geohash_ubsan.sql diff --git a/src/Functions/GeoHash.cpp b/src/Functions/GeoHash.cpp index 3ebc6f3d0fc..595bcacd41a 100644 --- a/src/Functions/GeoHash.cpp +++ b/src/Functions/GeoHash.cpp @@ -216,9 +216,7 @@ inline Float64 getSpan(uint8_t precision, CoordType type) inline uint8_t geohashPrecision(uint8_t precision) { if (precision == 0 || precision > MAX_PRECISION) - { precision = MAX_PRECISION; - } return precision; } @@ -281,13 +279,21 @@ GeohashesInBoxPreparedArgs geohashesInBoxPrepare( return {}; } - longitude_min = std::max(longitude_min, LON_MIN); - longitude_max = std::min(longitude_max, LON_MAX); - latitude_min = std::max(latitude_min, LAT_MIN); - latitude_max = std::min(latitude_max, LAT_MAX); + auto saturate = [](Float64 & value, Float64 min, Float64 max) + { + if (value < min) + value = min; + else if (value > max) + value = max; + }; - const auto lon_step = getSpan(precision, LONGITUDE); - const auto lat_step = getSpan(precision, LATITUDE); + saturate(longitude_min, LON_MIN, LON_MAX); + saturate(longitude_max, LON_MIN, LON_MAX); + saturate(latitude_min, LAT_MIN, LAT_MAX); + saturate(latitude_max, LAT_MIN, LAT_MAX); + + Float64 lon_step = getSpan(precision, LONGITUDE); + Float64 lat_step = getSpan(precision, LATITUDE); /// Align max to the right (or up) border of geohash grid cell to ensure that cell is in result. Float64 lon_min = floor(longitude_min / lon_step) * lon_step; diff --git a/tests/queries/0_stateless/01684_geohash_ubsan.reference b/tests/queries/0_stateless/01684_geohash_ubsan.reference new file mode 100644 index 00000000000..2ae4be53dea --- /dev/null +++ b/tests/queries/0_stateless/01684_geohash_ubsan.reference @@ -0,0 +1 @@ +['ypzpgxczgpyr'] diff --git a/tests/queries/0_stateless/01684_geohash_ubsan.sql b/tests/queries/0_stateless/01684_geohash_ubsan.sql new file mode 100644 index 00000000000..e7eb9c526b4 --- /dev/null +++ b/tests/queries/0_stateless/01684_geohash_ubsan.sql @@ -0,0 +1 @@ +SELECT geohashesInBox(100.0000991821289, 100.0000991821289, 1000.0001220703125, 1000.0001220703125, 0); From 5f7a852b921f03fce00111f671edc55fb9eda856 Mon Sep 17 00:00:00 2001 From: PHO Date: Tue, 2 Feb 2021 12:36:40 +0900 Subject: [PATCH 89/94] Fix a segfault in fromModifiedJulianDay() It was crashing when the argument type was Nullable(T) where T was any integral type other than Int32. --- src/Functions/fromModifiedJulianDay.cpp | 4 ++-- .../queries/0_stateless/01544_fromModifiedJulianDay.reference | 1 + tests/queries/0_stateless/01544_fromModifiedJulianDay.sql | 1 + 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Functions/fromModifiedJulianDay.cpp b/src/Functions/fromModifiedJulianDay.cpp index 636512db0de..cd5699bfac5 100644 --- a/src/Functions/fromModifiedJulianDay.cpp +++ b/src/Functions/fromModifiedJulianDay.cpp @@ -163,7 +163,7 @@ namespace DB FunctionBaseImplPtr build(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override { - const DataTypePtr & from_type = arguments[0].type; + const DataTypePtr & from_type = removeNullable(arguments[0].type); DataTypes argument_types = { from_type }; FunctionBaseImplPtr base; auto call = [&](const auto & types) -> bool @@ -185,7 +185,7 @@ namespace DB * here causes a SEGV. So we must somehow create a * dummy implementation and return it. */ - if (WhichDataType(from_type).isNullable()) // Nullable(Nothing) + if (WhichDataType(from_type).isNothing()) // Nullable(Nothing) return std::make_unique>(argument_types, return_type); else // Should not happen. diff --git a/tests/queries/0_stateless/01544_fromModifiedJulianDay.reference b/tests/queries/0_stateless/01544_fromModifiedJulianDay.reference index 6f71b6263c0..443b90b80a5 100644 --- a/tests/queries/0_stateless/01544_fromModifiedJulianDay.reference +++ b/tests/queries/0_stateless/01544_fromModifiedJulianDay.reference @@ -3,6 +3,7 @@ Invocation with constant 1858-11-17 2020-11-01 \N +\N or null 2020-11-01 \N diff --git a/tests/queries/0_stateless/01544_fromModifiedJulianDay.sql b/tests/queries/0_stateless/01544_fromModifiedJulianDay.sql index 4e50351d191..5e682a942d5 100644 --- a/tests/queries/0_stateless/01544_fromModifiedJulianDay.sql +++ b/tests/queries/0_stateless/01544_fromModifiedJulianDay.sql @@ -5,6 +5,7 @@ SELECT fromModifiedJulianDay(-1); SELECT fromModifiedJulianDay(0); SELECT fromModifiedJulianDay(59154); SELECT fromModifiedJulianDay(NULL); +SELECT fromModifiedJulianDay(CAST(NULL, 'Nullable(Int64)')); SELECT fromModifiedJulianDay(-678942); -- { serverError 490 } SELECT fromModifiedJulianDay(2973484); -- { serverError 490 } From 656cd583f75fef2d50144f506d68749e107fd8dd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 2 Feb 2021 06:46:54 +0300 Subject: [PATCH 90/94] Add MSan annotation for system.stack_trace --- src/Common/StackTrace.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Common/StackTrace.cpp b/src/Common/StackTrace.cpp index b285a45bdc5..44f6b9e5443 100644 --- a/src/Common/StackTrace.cpp +++ b/src/Common/StackTrace.cpp @@ -261,6 +261,9 @@ StackTrace::StackTrace(const ucontext_t & signal_context) { tryCapture(); + /// This variable from signal handler is not instrumented by Memory Sanitizer. + __msan_unpoison(&signal_context, sizeof(signal_context)); + void * caller_address = getCallerAddress(signal_context); if (size == 0 && caller_address) From 8c0ec5105b67c73799a6d8774d7bef939ea2668c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Tue, 2 Feb 2021 06:53:11 +0300 Subject: [PATCH 91/94] Add a patch from @FishermanZzhang, #19952 --- programs/copier/ClusterCopier.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp index ca09e7c1889..d44e24dca49 100644 --- a/programs/copier/ClusterCopier.cpp +++ b/programs/copier/ClusterCopier.cpp @@ -642,7 +642,7 @@ TaskStatus ClusterCopier::tryMoveAllPiecesToDestinationTable(const TaskTable & t query_deduplicate_ast_string += " OPTIMIZE TABLE " + getQuotedTable(original_table) + ((partition_name == "'all'") ? " PARTITION ID " : " PARTITION ") + partition_name + " DEDUPLICATE;"; - LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: {}", query_alter_ast_string); + LOG_DEBUG(log, "Executing OPTIMIZE DEDUPLICATE query: {}", query_deduplicate_ast_string); UInt64 num_nodes = executeQueryOnCluster( task_table.cluster_push, From 078011ef2d15258b1f23b8b1b794e888b3078975 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 2 Feb 2021 07:16:47 +0300 Subject: [PATCH 92/94] Allow conversion from double to float in function JSONExtract beacuse the users want that --- src/Functions/FunctionsJSON.h | 12 +++++++++- ...685_json_extract_double_as_float.reference | 10 ++++++++ .../01685_json_extract_double_as_float.sql | 24 +++++++++++++++++++ 3 files changed, 45 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01685_json_extract_double_as_float.reference create mode 100644 tests/queries/0_stateless/01685_json_extract_double_as_float.sql diff --git a/src/Functions/FunctionsJSON.h b/src/Functions/FunctionsJSON.h index aea5829eaef..f066bb1029a 100644 --- a/src/Functions/FunctionsJSON.h +++ b/src/Functions/FunctionsJSON.h @@ -25,6 +25,7 @@ #include #include #include +#include #include #if !defined(ARCADIA_BUILD) @@ -507,11 +508,20 @@ public: } else if (element.isDouble()) { - if (!accurate::convertNumeric(element.getDouble(), value)) + if constexpr (std::is_floating_point_v) + { + /// We permit inaccurate conversion of double to float. + /// Example: double 0.1 from JSON is not representable in float. + /// But it will be more convenient for user to perform conversion. + value = element.getDouble(); + } + else if (!accurate::convertNumeric(element.getDouble(), value)) return false; } else if (element.isBool() && is_integer_v && convert_bool_to_integer) + { value = static_cast(element.getBool()); + } else return false; diff --git a/tests/queries/0_stateless/01685_json_extract_double_as_float.reference b/tests/queries/0_stateless/01685_json_extract_double_as_float.reference new file mode 100644 index 00000000000..f3f4206b425 --- /dev/null +++ b/tests/queries/0_stateless/01685_json_extract_double_as_float.reference @@ -0,0 +1,10 @@ +1.1 1.1 1.1 1.1 +0.01 0.01 0.01 0.01 +0 +\N +-1e300 +-inf +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/01685_json_extract_double_as_float.sql b/tests/queries/0_stateless/01685_json_extract_double_as_float.sql new file mode 100644 index 00000000000..c9aa2518085 --- /dev/null +++ b/tests/queries/0_stateless/01685_json_extract_double_as_float.sql @@ -0,0 +1,24 @@ +WITH '{ "v":1.1}' AS raw +SELECT + JSONExtract(raw, 'v', 'float') AS float32_1, + JSONExtract(raw, 'v', 'Float32') AS float32_2, + JSONExtractFloat(raw, 'v') AS float64_1, + JSONExtract(raw, 'v', 'double') AS float64_2; + +WITH '{ "v":1E-2}' AS raw +SELECT + JSONExtract(raw, 'v', 'float') AS float32_1, + JSONExtract(raw, 'v', 'Float32') AS float32_2, + JSONExtractFloat(raw, 'v') AS float64_1, + JSONExtract(raw, 'v', 'double') AS float64_2; + +SELECT JSONExtract('{"v":1.1}', 'v', 'UInt64'); +SELECT JSONExtract('{"v":1.1}', 'v', 'Nullable(UInt64)'); + +SELECT JSONExtract('{"v":-1e300}', 'v', 'Float64'); +SELECT JSONExtract('{"v":-1e300}', 'v', 'Float32'); + +SELECT JSONExtract('{"v":-1e300}', 'v', 'UInt64'); +SELECT JSONExtract('{"v":-1e300}', 'v', 'Int64'); +SELECT JSONExtract('{"v":-1e300}', 'v', 'UInt8'); +SELECT JSONExtract('{"v":-1e300}', 'v', 'Int8'); From 3bc7a64abe65498eac3105622b9bb48e637ed7eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 2 Feb 2021 08:27:07 +0300 Subject: [PATCH 93/94] Fix extremely inconvenient Markdown markup in Adopters page --- docs/en/introduction/adopters.md | 222 +++++++++++++++---------------- 1 file changed, 111 insertions(+), 111 deletions(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 2684e6fdd3a..707a05b63e5 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -8,118 +8,118 @@ toc_title: Adopters !!! warning "Disclaimer" The following list of companies using ClickHouse and their success stories is assembled from public sources, thus might differ from current reality. We’d appreciate it if you share the story of adopting ClickHouse in your company and [add it to the list](https://github.com/ClickHouse/ClickHouse/edit/master/docs/en/introduction/adopters.md), but please make sure you won’t have any NDA issues by doing so. Providing updates with publications from other companies is also useful. -| Company | Industry | Usecase | Cluster Size | (Un)Compressed Data Size\* | Reference | -|------------------------------------------------------------------------------------------------|---------------------------------|-----------------------|------------------------------------------------------------|------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| 2gis | Maps | Monitoring | — | — | [Talk in Russian, July 2019](https://youtu.be/58sPkXfq6nw) | -| Admiral | Martech | Engagement Management | — | — | [Webinar Slides, June 2020](https://altinity.com/presentations/2020/06/16/big-data-in-real-time-how-clickhouse-powers-admirals-visitor-relationships-for-publishers) | -| Alibaba Cloud | Cloud | Managed Service | — | — | [Official Website](https://help.aliyun.com/product/144466.html) | -| Aloha Browser | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://presentations.clickhouse.tech/meetup22/aloha.pdf) | -| Amadeus | Travel | Analytics | — | — | [Press Release, April 2018](https://www.altinity.com/blog/2018/4/5/amadeus-technologies-launches-investment-and-insights-tool-based-on-machine-learning-and-strategy-algorithms) | -| Appsflyer | Mobile analytics | Main product | — | — | [Talk in Russian, July 2019](https://www.youtube.com/watch?v=M3wbRlcpBbY) | -| ArenaData | Data Platform | Main product | — | — | [Slides in Russian, December 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup38/indexes.pdf) | -| Avito | Classifieds | Monitoring | — | — | [Meetup, April 2020](https://www.youtube.com/watch?v=n1tm4j4W8ZQ) | -| Badoo | Dating | Timeseries | — | — | [Slides in Russian, December 2019](https://presentations.clickhouse.tech/meetup38/forecast.pdf) | -| Benocs | Network Telemetry and Analytics | Main Product | — | — | [Slides in English, October 2017](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup9/lpm.pdf) | +| Company | Industry | Usecase | Cluster Size | (Un)Compressed Data Size\* | Reference | +|---------|----------|---------|--------------|------------------------------------------------------------------------------|-----------| +| 2gis | Maps | Monitoring | — | — | [Talk in Russian, July 2019](https://youtu.be/58sPkXfq6nw) | +| Admiral | Martech | Engagement Management | — | — | [Webinar Slides, June 2020](https://altinity.com/presentations/2020/06/16/big-data-in-real-time-how-clickhouse-powers-admirals-visitor-relationships-for-publishers) | +| Alibaba Cloud | Cloud | Managed Service | — | — | [Official Website](https://help.aliyun.com/product/144466.html) | +| Aloha Browser | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://presentations.clickhouse.tech/meetup22/aloha.pdf) | +| Amadeus | Travel | Analytics | — | — | [Press Release, April 2018](https://www.altinity.com/blog/2018/4/5/amadeus-technologies-launches-investment-and-insights-tool-based-on-machine-learning-and-strategy-algorithms) | +| Appsflyer | Mobile analytics | Main product | — | — | [Talk in Russian, July 2019](https://www.youtube.com/watch?v=M3wbRlcpBbY) | +| ArenaData | Data Platform | Main product | — | — | [Slides in Russian, December 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup38/indexes.pdf) | +| Avito | Classifieds | Monitoring | — | — | [Meetup, April 2020](https://www.youtube.com/watch?v=n1tm4j4W8ZQ) | +| Badoo | Dating | Timeseries | — | — | [Slides in Russian, December 2019](https://presentations.clickhouse.tech/meetup38/forecast.pdf) | +| Benocs | Network Telemetry and Analytics | Main Product | — | — | [Slides in English, October 2017](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup9/lpm.pdf) | | BIGO | Video | Computing Platform | — | — | [Blog Article, August 2020](https://www.programmersought.com/article/44544895251/) | -| Bloomberg | Finance, Media | Monitoring | 102 servers | — | [Slides, May 2018](https://www.slideshare.net/Altinity/http-analytics-for-6m-requests-per-second-using-clickhouse-by-alexander-bocharov) | -| Bloxy | Blockchain | Analytics | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/4_bloxy.pptx) | -| Bytedance | Social platforms | — | — | — | [The ClickHouse Meetup East, October 2020](https://www.youtube.com/watch?v=ckChUkC3Pns) | +| Bloomberg | Finance, Media | Monitoring | 102 servers | — | [Slides, May 2018](https://www.slideshare.net/Altinity/http-analytics-for-6m-requests-per-second-using-clickhouse-by-alexander-bocharov) | +| Bloxy | Blockchain | Analytics | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/4_bloxy.pptx) | +| Bytedance | Social platforms | — | — | — | [The ClickHouse Meetup East, October 2020](https://www.youtube.com/watch?v=ckChUkC3Pns) | | CardsMobile | Finance | Analytics | — | — | [VC.ru](https://vc.ru/s/cardsmobile/143449-rukovoditel-gruppy-analiza-dannyh) | -| CARTO | Business Intelligence | Geo analytics | — | — | [Geospatial processing with ClickHouse](https://carto.com/blog/geospatial-processing-with-clickhouse/) | -| CERN | Research | Experiment | — | — | [Press release, April 2012](https://www.yandex.com/company/press_center/press_releases/2012/2012-04-10/) | -| Cisco | Networking | Traffic analysis | — | — | [Lightning talk, October 2019](https://youtu.be/-hI1vDR2oPY?t=5057) | -| Citadel Securities | Finance | — | — | — | [Contribution, March 2019](https://github.com/ClickHouse/ClickHouse/pull/4774) | -| Citymobil | Taxi | Analytics | — | — | [Blog Post in Russian, March 2020](https://habr.com/en/company/citymobil/blog/490660/) | -| Cloudflare | CDN | Traffic analysis | 36 servers | — | [Blog post, May 2017](https://blog.cloudflare.com/how-cloudflare-analyzes-1m-dns-queries-per-second/), [Blog post, March 2018](https://blog.cloudflare.com/http-analytics-for-6m-requests-per-second-using-clickhouse/) | -| Comcast | Media | CDN Traffic Analysis | — | — | [ApacheCon 2019 Talk](https://www.youtube.com/watch?v=e9TZ6gFDjNg) | -| ContentSquare | Web analytics | Main product | — | — | [Blog post in French, November 2018](http://souslecapot.net/2018/11/21/patrick-chatain-vp-engineering-chez-contentsquare-penser-davantage-amelioration-continue-que-revolution-constante/) | -| Corunet | Analytics | Main product | — | — | [Slides in English, April 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup21/predictive_models.pdf) | -| CraiditX 氪信 | Finance AI | Analysis | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/udf.pptx) | -| Crazypanda | Games | | — | — | Live session on ClickHouse meetup | -| Criteo | Retail | Main product | — | — | [Slides in English, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup18/3_storetail.pptx) | -| Dataliance for China Telecom | Telecom | Analytics | — | — | [Slides in Chinese, January 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup12/telecom.pdf) | -| Deutsche Bank | Finance | BI Analytics | — | — | [Slides in English, October 2019](https://bigdatadays.ru/wp-content/uploads/2019/10/D2-H3-3_Yakunin-Goihburg.pdf) | -| Deeplay | Gaming Analytics | — | — | — | [Job advertisement, 2020](https://career.habr.com/vacancies/1000062568) | -| Diva-e | Digital consulting | Main Product | — | — | [Slides in English, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup29/ClickHouse-MeetUp-Unusual-Applications-sd-2019-09-17.pdf) | -| Ecwid | E-commerce SaaS | Metrics, Logging | — | — | [Slides in Russian, April 2019](https://nastachku.ru/var/files/1/presentation/backend/2_Backend_6.pdf) | -| eBay | E-commerce | Logs, Metrics and Events | — | — | [Official website, Sep 2020](https://tech.ebayinc.com/engineering/ou-online-analytical-processing/) | -| Exness | Trading | Metrics, Logging | — | — | [Talk in Russian, May 2019](https://youtu.be/_rpU-TvSfZ8?t=3215) | -| FastNetMon | DDoS Protection | Main Product | | — | [Official website](https://fastnetmon.com/docs-fnm-advanced/fastnetmon-advanced-traffic-persistency/) | -| Flipkart | e-Commerce | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=239) | -| FunCorp | Games | | — | — | [Article](https://www.altinity.com/blog/migrating-from-redshift-to-clickhouse) | -| Geniee | Ad network | Main product | — | — | [Blog post in Japanese, July 2017](https://tech.geniee.co.jp/entry/2017/07/20/160100) | -| Genotek | Bioinformatics | Main product | — | — | [Video, August 2020](https://youtu.be/v3KyZbz9lEE) | -| HUYA | Video Streaming | Analytics | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/7.%20ClickHouse万亿数据分析实践%20李本旺(sundy-li)%20虎牙.pdf) | -| ICA | FinTech | Risk Management | — | — | [Blog Post in English, Sep 2020](https://altinity.com/blog/clickhouse-vs-redshift-performance-for-fintech-risk-management?utm_campaign=ClickHouse%20vs%20RedShift&utm_content=143520807&utm_medium=social&utm_source=twitter&hss_channel=tw-3894792263) | -| Idealista | Real Estate | Analytics | — | — | [Blog Post in English, April 2019](https://clickhouse.tech/blog/en/clickhouse-meetup-in-madrid-on-april-2-2019) | -| Infovista | Networks | Analytics | — | — | [Slides in English, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup30/infovista.pdf) | -| InnoGames | Games | Metrics, Logging | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/graphite_and_clickHouse.pdf) | -| Instana | APM Platform | Main product | — | — | [Twitter post](https://twitter.com/mieldonkers/status/1248884119158882304) | -| Integros | Platform for video services | Analytics | — | — | [Slides in Russian, May 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup22/strategies.pdf) | -| Ippon Technologies | Technology Consulting | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=205) | -| Ivi | Online Cinema | Analytics, Monitoring | — | — | [Article in Russian, Jan 2018](https://habr.com/en/company/ivi/blog/347408/) | -| Jinshuju 金数据 | BI Analytics | Main product | — | — | [Slides in Chinese, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/3.%20金数据数据架构调整方案Public.pdf) | -| Kodiak Data | Clouds | Main product | — | — | [Slides in Engish, April 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup13/kodiak_data.pdf) | -| Kontur | Software Development | Metrics | — | — | [Talk in Russian, November 2018](https://www.youtube.com/watch?v=U4u4Bd0FtrY) | -| Kuaishou | Video | — | — | — | [ClickHouse Meetup, October 2018](https://clickhouse.tech/blog/en/2018/clickhouse-community-meetup-in-beijing-on-october-28-2018/) | -| Lawrence Berkeley National Laboratory | Research | Traffic analysis | 1 server | 11.8 TiB | [Slides in English, April 2019](https://www.smitasin.com/presentations/2019-04-17_DOE-NSM.pdf) | -| LifeStreet | Ad network | Main product | 75 servers (3 replicas) | 5.27 PiB | [Blog post in Russian, February 2017](https://habr.com/en/post/322620/) | -| Mail.ru Cloud Solutions | Cloud services | Main product | — | — | [Article in Russian](https://mcs.mail.ru/help/db-create/clickhouse#) | -| Marilyn | Advertising | Statistics | — | — | [Talk in Russian, June 2017](https://www.youtube.com/watch?v=iXlIgx2khwc) | -| Mello | Marketing | Analytics | 1 server | — | [Article, Oct 2020](https://vc.ru/marketing/166180-razrabotka-tipovogo-otcheta-skvoznoy-analitiki) | -| MessageBird | Telecommunications | Statistics | — | — | [Slides in English, November 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup20/messagebird.pdf) | -| MindsDB | Machine Learning | Main Product | — | — | [Official Website](https://www.mindsdb.com/blog/machine-learning-models-as-tables-in-ch) |x -| MUX | Online Video | Video Analytics | — | — | [Talk in English, August 2019](https://altinity.com/presentations/2019/8/13/how-clickhouse-became-the-default-analytics-database-for-mux/) | -| MGID | Ad network | Web-analytics | — | — | [Blog post in Russian, April 2020](http://gs-studio.com/news-about-it/32777----clickhouse---c) | -| NOC Project | Network Monitoring | Analytics | Main Product | — | [Official Website](https://getnoc.com/features/big-data/) | -| Nuna Inc. | Health Data Analytics | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=170) | -| OneAPM | Monitorings and Data Analysis | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/8.%20clickhouse在OneAPM的应用%20杜龙.pdf) | -| Percent 百分点 | Analytics | Main Product | — | — | [Slides in Chinese, June 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/4.%20ClickHouse万亿数据双中心的设计与实践%20.pdf) | -| Percona | Performance analysis | Percona Monitoring and Management | — | — | [Official website, Mar 2020](https://www.percona.com/blog/2020/03/30/advanced-query-analysis-in-percona-monitoring-and-management-with-direct-clickhouse-access/) | -| Plausible | Analytics | Main Product | — | — | [Blog post, June 2020](https://twitter.com/PlausibleHQ/status/1273889629087969280) | -| PostHog | Product Analytics | Main Product | — | — | [Release Notes, Oct 2020](https://posthog.com/blog/the-posthog-array-1-15-0) | -| Postmates | Delivery | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=188) | -| Pragma Innovation | Telemetry and Big Data Analysis | Main product | — | — | [Slides in English, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup18/4_pragma_innovation.pdf) | -| QINGCLOUD | Cloud services | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/4.%20Cloud%20%2B%20TSDB%20for%20ClickHouse%20张健%20QingCloud.pdf) | -| Qrator | DDoS protection | Main product | — | — | [Blog Post, March 2019](https://blog.qrator.net/en/clickhouse-ddos-mitigation_37/) | -| Raiffeisenbank | Banking | Analytics | — | — | [Lecture in Russian, December 2020](https://cs.hse.ru/announcements/421965599.html) | -| Rambler | Internet services | Analytics | — | — | [Talk in Russian, April 2018](https://medium.com/@ramblertop/разработка-api-clickhouse-для-рамблер-топ-100-f4c7e56f3141) | -| Retell | Speech synthesis | Analytics | — | — | [Blog Article, August 2020](https://vc.ru/services/153732-kak-sozdat-audiostati-na-vashem-sayte-i-zachem-eto-nuzhno) | -| Rspamd | Antispam | Analytics | — | — | [Official Website](https://rspamd.com/doc/modules/clickhouse.html) | -| RuSIEM | SIEM | Main Product | — | — | [Official Website](https://rusiem.com/en/products/architecture) | -| S7 Airlines | Airlines | Metrics, Logging | — | — | [Talk in Russian, March 2019](https://www.youtube.com/watch?v=nwG68klRpPg&t=15s) | -| scireum GmbH | e-Commerce | Main product | — | — | [Talk in German, February 2020](https://www.youtube.com/watch?v=7QWAn5RbyR4) | -| Segment | Data processing | Main product | 9 * i3en.3xlarge nodes 7.5TB NVME SSDs, 96GB Memory, 12 vCPUs | — | [Slides, 2019](https://slides.com/abraithwaite/segment-clickhouse) | -| SEMrush | Marketing | Main product | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/5_semrush.pdf) | -| Sentry | Software Development | Main product | — | — | [Blog Post in English, May 2019](https://blog.sentry.io/2019/05/16/introducing-snuba-sentrys-new-search-infrastructure) | -| seo.do | Analytics | Main product | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/CH%20Presentation-%20Metehan%20Çetinkaya.pdf) | -| SGK | Goverment Social Security | Analytics | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/ClickHouse%20Meetup-Ramazan%20POLAT.pdf) | -| Sina | News | — | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/6.%20ClickHouse最佳实践%20高鹏_新浪.pdf) | -| SMI2 | News | Analytics | — | — | [Blog Post in Russian, November 2017](https://habr.com/ru/company/smi2/blog/314558/) | -| Splunk | Business Analytics | Main product | — | — | [Slides in English, January 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup12/splunk.pdf) | -| Spotify | Music | Experimentation | — | — | [Slides, July 2018](https://www.slideshare.net/glebus/using-clickhouse-for-experimentation-104247173) | -| Staffcop | Information Security | Main Product | — | — | [Official website, Documentation](https://www.staffcop.ru/sce43) | -| Suning | E-Commerce | User behaviour analytics | — | — | [Blog article](https://www.sohu.com/a/434152235_411876) | -| Teralytics | Mobility | Analytics | — | — | [Tech blog](https://www.teralytics.net/knowledge-hub/visualizing-mobility-data-the-scalability-challenge) | -| Tencent | Big Data | Data processing | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/5.%20ClickHouse大数据集群应用_李俊飞腾讯网媒事业部.pdf) | -| Tencent | Messaging | Logging | — | — | [Talk in Chinese, November 2019](https://youtu.be/T-iVQRuw-QY?t=5050) | -| Tencent Music Entertainment (TME) | BigData | Data processing | — | — | [Blog in Chinese, June 2020](https://cloud.tencent.com/developer/article/1637840) | -| Traffic Stars | AD network | — | — | — | [Slides in Russian, May 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup15/lightning/ninja.pdf) | -| Uber | Taxi | Logging | — | — | [Slides, February 2020](https://presentations.clickhouse.tech/meetup40/uber.pdf) | -| VKontakte | Social Network | Statistics, Logging | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/3_vk.pdf) | -| Walmart Labs | Internet, Retail | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=144) | -| Wargaming | Games | | — | — | [Interview](https://habr.com/en/post/496954/) | -| Wisebits | IT Solutions | Analytics | — | — | [Slides in Russian, May 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup22/strategies.pdf) | -| Workato | Automation Software | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=334) | -| Xiaoxin Tech | Education | Common purpose | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/sync-clickhouse-with-mysql-mongodb.pptx) | -| Ximalaya | Audio sharing | OLAP | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/ximalaya.pdf) | -| Yandex Cloud | Public Cloud | Main product | — | — | [Talk in Russian, December 2019](https://www.youtube.com/watch?v=pgnak9e_E0o) | -| Yandex DataLens | Business Intelligence | Main product | — | — | [Slides in Russian, December 2019](https://presentations.clickhouse.tech/meetup38/datalens.pdf) | -| Yandex Market | e-Commerce | Metrics, Logging | — | — | [Talk in Russian, January 2019](https://youtu.be/_l1qP0DyBcA?t=478) | -| Yandex Metrica | Web analytics | Main product | 630 servers in one cluster, 360 servers in another cluster, 1862 servers in one department | 133 PiB / 8.31 PiB / 120 trillion records | [Slides, February 2020](https://presentations.clickhouse.tech/meetup40/introduction/#13) | -| ЦВТ | Software Development | Metrics, Logging | — | — | [Blog Post, March 2019, in Russian](https://vc.ru/dev/62715-kak-my-stroili-monitoring-na-prometheus-clickhouse-i-elk) | -| МКБ | Bank | Web-system monitoring | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/mkb.pdf) | -| ЦФТ | Banking, Financial products, Payments | — | — | — | [Meetup in Russian, April 2020](https://team.cft.ru/events/162) | -| kakaocorp | Internet company | — | — | — | [if(kakao)2020 conference](https://if.kakao.com/session/117) | +| CARTO | Business Intelligence | Geo analytics | — | — | [Geospatial processing with ClickHouse](https://carto.com/blog/geospatial-processing-with-clickhouse/) | +| CERN | Research | Experiment | — | — | [Press release, April 2012](https://www.yandex.com/company/press_center/press_releases/2012/2012-04-10/) | +| Cisco | Networking | Traffic analysis | — | — | [Lightning talk, October 2019](https://youtu.be/-hI1vDR2oPY?t=5057) | +| Citadel Securities | Finance | — | — | — | [Contribution, March 2019](https://github.com/ClickHouse/ClickHouse/pull/4774) | +| Citymobil | Taxi | Analytics | — | — | [Blog Post in Russian, March 2020](https://habr.com/en/company/citymobil/blog/490660/) | +| Cloudflare | CDN | Traffic analysis | 36 servers | — | [Blog post, May 2017](https://blog.cloudflare.com/how-cloudflare-analyzes-1m-dns-queries-per-second/), [Blog post, March 2018](https://blog.cloudflare.com/http-analytics-for-6m-requests-per-second-using-clickhouse/) | +| Comcast | Media | CDN Traffic Analysis | — | — | [ApacheCon 2019 Talk](https://www.youtube.com/watch?v=e9TZ6gFDjNg) | +| ContentSquare | Web analytics | Main product | — | — | [Blog post in French, November 2018](http://souslecapot.net/2018/11/21/patrick-chatain-vp-engineering-chez-contentsquare-penser-davantage-amelioration-continue-que-revolution-constante/) | +| Corunet | Analytics | Main product | — | — | [Slides in English, April 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup21/predictive_models.pdf) | +| CraiditX 氪信 | Finance AI | Analysis | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/udf.pptx) | +| Crazypanda | Games | | — | — | Live session on ClickHouse meetup | +| Criteo | Retail | Main product | — | — | [Slides in English, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup18/3_storetail.pptx) | +| Dataliance for China Telecom | Telecom | Analytics | — | — | [Slides in Chinese, January 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup12/telecom.pdf) | +| Deutsche Bank | Finance | BI Analytics | — | — | [Slides in English, October 2019](https://bigdatadays.ru/wp-content/uploads/2019/10/D2-H3-3_Yakunin-Goihburg.pdf) | +| Deeplay | Gaming Analytics | — | — | — | [Job advertisement, 2020](https://career.habr.com/vacancies/1000062568) | +| Diva-e | Digital consulting | Main Product | — | — | [Slides in English, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup29/ClickHouse-MeetUp-Unusual-Applications-sd-2019-09-17.pdf) | +| Ecwid | E-commerce SaaS | Metrics, Logging | — | — | [Slides in Russian, April 2019](https://nastachku.ru/var/files/1/presentation/backend/2_Backend_6.pdf) | +| eBay | E-commerce | Logs, Metrics and Events | — | — | [Official website, Sep 2020](https://tech.ebayinc.com/engineering/ou-online-analytical-processing/) | +| Exness | Trading | Metrics, Logging | — | — | [Talk in Russian, May 2019](https://youtu.be/_rpU-TvSfZ8?t=3215) | +| FastNetMon | DDoS Protection | Main Product | | — | [Official website](https://fastnetmon.com/docs-fnm-advanced/fastnetmon-advanced-traffic-persistency/) | +| Flipkart | e-Commerce | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=239) | +| FunCorp | Games | | — | — | [Article](https://www.altinity.com/blog/migrating-from-redshift-to-clickhouse) | +| Geniee | Ad network | Main product | — | — | [Blog post in Japanese, July 2017](https://tech.geniee.co.jp/entry/2017/07/20/160100) | +| Genotek | Bioinformatics | Main product | — | — | [Video, August 2020](https://youtu.be/v3KyZbz9lEE) | +| HUYA | Video Streaming | Analytics | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/7.%20ClickHouse万亿数据分析实践%20李本旺(sundy-li)%20虎牙.pdf) | +| ICA | FinTech | Risk Management | — | — | [Blog Post in English, Sep 2020](https://altinity.com/blog/clickhouse-vs-redshift-performance-for-fintech-risk-management?utm_campaign=ClickHouse%20vs%20RedShift&utm_content=143520807&utm_medium=social&utm_source=twitter&hss_channel=tw-3894792263) | +| Idealista | Real Estate | Analytics | — | — | [Blog Post in English, April 2019](https://clickhouse.tech/blog/en/clickhouse-meetup-in-madrid-on-april-2-2019) | +| Infovista | Networks | Analytics | — | — | [Slides in English, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup30/infovista.pdf) | +| InnoGames | Games | Metrics, Logging | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/graphite_and_clickHouse.pdf) | +| Instana | APM Platform | Main product | — | — | [Twitter post](https://twitter.com/mieldonkers/status/1248884119158882304) | +| Integros | Platform for video services | Analytics | — | — | [Slides in Russian, May 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup22/strategies.pdf) | +| Ippon Technologies | Technology Consulting | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=205) | +| Ivi | Online Cinema | Analytics, Monitoring | — | — | [Article in Russian, Jan 2018](https://habr.com/en/company/ivi/blog/347408/) | +| Jinshuju 金数据 | BI Analytics | Main product | — | — | [Slides in Chinese, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/3.%20金数据数据架构调整方案Public.pdf) | +| Kodiak Data | Clouds | Main product | — | — | [Slides in Engish, April 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup13/kodiak_data.pdf) | +| Kontur | Software Development | Metrics | — | — | [Talk in Russian, November 2018](https://www.youtube.com/watch?v=U4u4Bd0FtrY) | +| Kuaishou | Video | — | — | — | [ClickHouse Meetup, October 2018](https://clickhouse.tech/blog/en/2018/clickhouse-community-meetup-in-beijing-on-october-28-2018/) | +| Lawrence Berkeley National Laboratory | Research | Traffic analysis | 1 server | 11.8 TiB | [Slides in English, April 2019](https://www.smitasin.com/presentations/2019-04-17_DOE-NSM.pdf) | +| LifeStreet | Ad network | Main product | 75 servers (3 replicas) | 5.27 PiB | [Blog post in Russian, February 2017](https://habr.com/en/post/322620/) | +| Mail.ru Cloud Solutions | Cloud services | Main product | — | — | [Article in Russian](https://mcs.mail.ru/help/db-create/clickhouse#) | +| Marilyn | Advertising | Statistics | — | — | [Talk in Russian, June 2017](https://www.youtube.com/watch?v=iXlIgx2khwc) | +| Mello | Marketing | Analytics | 1 server | — | [Article, Oct 2020](https://vc.ru/marketing/166180-razrabotka-tipovogo-otcheta-skvoznoy-analitiki) | +| MessageBird | Telecommunications | Statistics | — | — | [Slides in English, November 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup20/messagebird.pdf) | +| MindsDB | Machine Learning | Main Product | — | — | [Official Website](https://www.mindsdb.com/blog/machine-learning-models-as-tables-in-ch) |x +| MUX | Online Video | Video Analytics | — | — | [Talk in English, August 2019](https://altinity.com/presentations/2019/8/13/how-clickhouse-became-the-default-analytics-database-for-mux/) | +| MGID | Ad network | Web-analytics | — | — | [Blog post in Russian, April 2020](http://gs-studio.com/news-about-it/32777----clickhouse---c) | +| NOC Project | Network Monitoring | Analytics | Main Product | — | [Official Website](https://getnoc.com/features/big-data/) | +| Nuna Inc. | Health Data Analytics | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=170) | +| OneAPM | Monitorings and Data Analysis | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/8.%20clickhouse在OneAPM的应用%20杜龙.pdf) | +| Percent 百分点 | Analytics | Main Product | — | — | [Slides in Chinese, June 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/4.%20ClickHouse万亿数据双中心的设计与实践%20.pdf) | +| Percona | Performance analysis | Percona Monitoring and Management | — | — | [Official website, Mar 2020](https://www.percona.com/blog/2020/03/30/advanced-query-analysis-in-percona-monitoring-and-management-with-direct-clickhouse-access/) | +| Plausible | Analytics | Main Product | — | — | [Blog post, June 2020](https://twitter.com/PlausibleHQ/status/1273889629087969280) | +| PostHog | Product Analytics | Main Product | — | — | [Release Notes, Oct 2020](https://posthog.com/blog/the-posthog-array-1-15-0) | +| Postmates | Delivery | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=188) | +| Pragma Innovation | Telemetry and Big Data Analysis | Main product | — | — | [Slides in English, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup18/4_pragma_innovation.pdf) | +| QINGCLOUD | Cloud services | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/4.%20Cloud%20%2B%20TSDB%20for%20ClickHouse%20张健%20QingCloud.pdf) | +| Qrator | DDoS protection | Main product | — | — | [Blog Post, March 2019](https://blog.qrator.net/en/clickhouse-ddos-mitigation_37/) | +| Raiffeisenbank | Banking | Analytics | — | — | [Lecture in Russian, December 2020](https://cs.hse.ru/announcements/421965599.html) | +| Rambler | Internet services | Analytics | — | — | [Talk in Russian, April 2018](https://medium.com/@ramblertop/разработка-api-clickhouse-для-рамблер-топ-100-f4c7e56f3141) | +| Retell | Speech synthesis | Analytics | — | — | [Blog Article, August 2020](https://vc.ru/services/153732-kak-sozdat-audiostati-na-vashem-sayte-i-zachem-eto-nuzhno) | +| Rspamd | Antispam | Analytics | — | — | [Official Website](https://rspamd.com/doc/modules/clickhouse.html) | +| RuSIEM | SIEM | Main Product | — | — | [Official Website](https://rusiem.com/en/products/architecture) | +| S7 Airlines | Airlines | Metrics, Logging | — | — | [Talk in Russian, March 2019](https://www.youtube.com/watch?v=nwG68klRpPg&t=15s) | +| scireum GmbH | e-Commerce | Main product | — | — | [Talk in German, February 2020](https://www.youtube.com/watch?v=7QWAn5RbyR4) | +| Segment | Data processing | Main product | 9 * i3en.3xlarge nodes 7.5TB NVME SSDs, 96GB Memory, 12 vCPUs | — | [Slides, 2019](https://slides.com/abraithwaite/segment-clickhouse) | +| SEMrush | Marketing | Main product | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/5_semrush.pdf) | +| Sentry | Software Development | Main product | — | — | [Blog Post in English, May 2019](https://blog.sentry.io/2019/05/16/introducing-snuba-sentrys-new-search-infrastructure) | +| seo.do | Analytics | Main product | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/CH%20Presentation-%20Metehan%20Çetinkaya.pdf) | +| SGK | Goverment Social Security | Analytics | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/ClickHouse%20Meetup-Ramazan%20POLAT.pdf) | +| Sina | News | — | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/6.%20ClickHouse最佳实践%20高鹏_新浪.pdf) | +| SMI2 | News | Analytics | — | — | [Blog Post in Russian, November 2017](https://habr.com/ru/company/smi2/blog/314558/) | +| Splunk | Business Analytics | Main product | — | — | [Slides in English, January 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup12/splunk.pdf) | +| Spotify | Music | Experimentation | — | — | [Slides, July 2018](https://www.slideshare.net/glebus/using-clickhouse-for-experimentation-104247173) | +| Staffcop | Information Security | Main Product | — | — | [Official website, Documentation](https://www.staffcop.ru/sce43) | +| Suning | E-Commerce | User behaviour analytics | — | — | [Blog article](https://www.sohu.com/a/434152235_411876) | +| Teralytics | Mobility | Analytics | — | — | [Tech blog](https://www.teralytics.net/knowledge-hub/visualizing-mobility-data-the-scalability-challenge) | +| Tencent | Big Data | Data processing | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/5.%20ClickHouse大数据集群应用_李俊飞腾讯网媒事业部.pdf) | +| Tencent | Messaging | Logging | — | — | [Talk in Chinese, November 2019](https://youtu.be/T-iVQRuw-QY?t=5050) | +| Tencent Music Entertainment (TME) | BigData | Data processing | — | — | [Blog in Chinese, June 2020](https://cloud.tencent.com/developer/article/1637840) | +| Traffic Stars | AD network | — | — | — | [Slides in Russian, May 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup15/lightning/ninja.pdf) | +| Uber | Taxi | Logging | — | — | [Slides, February 2020](https://presentations.clickhouse.tech/meetup40/uber.pdf) | +| VKontakte | Social Network | Statistics, Logging | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/3_vk.pdf) | +| Walmart Labs | Internet, Retail | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=144) | +| Wargaming | Games | | — | — | [Interview](https://habr.com/en/post/496954/) | +| Wisebits | IT Solutions | Analytics | — | — | [Slides in Russian, May 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup22/strategies.pdf) | +| Workato | Automation Software | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=334) | +| Xiaoxin Tech | Education | Common purpose | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/sync-clickhouse-with-mysql-mongodb.pptx) | +| Ximalaya | Audio sharing | OLAP | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/ximalaya.pdf) | +| Yandex Cloud | Public Cloud | Main product | — | — | [Talk in Russian, December 2019](https://www.youtube.com/watch?v=pgnak9e_E0o) | +| Yandex DataLens | Business Intelligence | Main product | — | — | [Slides in Russian, December 2019](https://presentations.clickhouse.tech/meetup38/datalens.pdf) | +| Yandex Market | e-Commerce | Metrics, Logging | — | — | [Talk in Russian, January 2019](https://youtu.be/_l1qP0DyBcA?t=478) | +| Yandex Metrica | Web analytics | Main product | 630 servers in one cluster, 360 servers in another cluster, 1862 servers in one department | 133 PiB / 8.31 PiB / 120 trillion records | [Slides, February 2020](https://presentations.clickhouse.tech/meetup40/introduction/#13) | +| ЦВТ | Software Development | Metrics, Logging | — | — | [Blog Post, March 2019, in Russian](https://vc.ru/dev/62715-kak-my-stroili-monitoring-na-prometheus-clickhouse-i-elk) | +| МКБ | Bank | Web-system monitoring | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/mkb.pdf) | +| ЦФТ | Banking, Financial products, Payments | — | — | — | [Meetup in Russian, April 2020](https://team.cft.ru/events/162) | +| kakaocorp | Internet company | — | — | — | [if(kakao)2020 conference](https://if.kakao.com/session/117) | [Original article](https://clickhouse.tech/docs/en/introduction/adopters/) From b9647e5326aaa4c138131e054b4d436e6a2389ad Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 2 Feb 2021 17:22:03 +0300 Subject: [PATCH 94/94] Fix double whitespace --- tests/performance/group_by_sundy_li.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/performance/group_by_sundy_li.xml b/tests/performance/group_by_sundy_li.xml index 3fcc4acf88d..dd512f7a53c 100644 --- a/tests/performance/group_by_sundy_li.xml +++ b/tests/performance/group_by_sundy_li.xml @@ -10,10 +10,10 @@ PARTITION BY d ORDER BY d - insert into a select '2000-01-01', ['aa','bb','cc','dd'][number % 4 + 1] from numbers(100000000) - insert into a select '2000-01-02', ['aa','bb','cc','dd'][number % 4 + 1] from numbers(100000000) - insert into a select '2000-01-03', ['aa','bb','cc','dd'][number % 4 + 1] from numbers(100000000) - insert into a select '2000-01-04', ['aa','bb','cc','dd'][number % 4 + 1] from numbers(100000000) + insert into a select '2000-01-01', ['aa','bb','cc','dd'][number % 4 + 1] from numbers(100000000) + insert into a select '2000-01-02', ['aa','bb','cc','dd'][number % 4 + 1] from numbers(100000000) + insert into a select '2000-01-03', ['aa','bb','cc','dd'][number % 4 + 1] from numbers(100000000) + insert into a select '2000-01-04', ['aa','bb','cc','dd'][number % 4 + 1] from numbers(100000000) OPTIMIZE TABLE a FINAL