From 5eeadb6c0704f5b41407ed6f26deac9d297e720f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 8 May 2024 10:55:26 +0200 Subject: [PATCH 01/43] Support running function for parameterized view value assignment --- src/Interpreters/Context.cpp | 4 ++-- src/Parsers/FunctionParameterValuesVisitor.cpp | 14 +++++++++++--- src/Parsers/FunctionParameterValuesVisitor.h | 3 ++- .../03146_parameterized_view_with_date.reference | 1 + .../03146_parameterized_view_with_date.sql | 12 ++++++++++++ 5 files changed, 28 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03146_parameterized_view_with_date.reference create mode 100644 tests/queries/0_stateless/03146_parameterized_view_with_date.sql diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 44d36e94441..4d84da36f78 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1840,7 +1840,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const if (table.get()->isView() && table->as() && table->as()->isParameterizedView()) { auto query = table->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone(); - NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression); + NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression, getQueryContext()); StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values); ASTCreateQuery create; @@ -2054,7 +2054,7 @@ StoragePtr Context::buildParametrizedViewStorage(const ASTPtr & table_expression return nullptr; auto query = original_view->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone(); - NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression); + NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression, getQueryContext()); StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values); ASTCreateQuery create; diff --git a/src/Parsers/FunctionParameterValuesVisitor.cpp b/src/Parsers/FunctionParameterValuesVisitor.cpp index 3692a4c73e5..e791e07cdfb 100644 --- a/src/Parsers/FunctionParameterValuesVisitor.cpp +++ b/src/Parsers/FunctionParameterValuesVisitor.cpp @@ -7,6 +7,7 @@ #include #include #include +#include namespace DB @@ -20,8 +21,9 @@ namespace ErrorCodes class FunctionParameterValuesVisitor { public: - explicit FunctionParameterValuesVisitor(NameToNameMap & parameter_values_) + explicit FunctionParameterValuesVisitor(NameToNameMap & parameter_values_, ContextPtr context_) : parameter_values(parameter_values_) + ,context(context_) { } @@ -35,6 +37,7 @@ public: private: NameToNameMap & parameter_values; + ContextPtr context; void visitFunction(const ASTFunction & parameter_function) { @@ -64,15 +67,20 @@ private: parameter_values[identifier->name()] = convertFieldToString(cast_literal->value); } } + else + { + ASTPtr res = evaluateConstantExpressionOrIdentifierAsLiteral(expression_list->children[1], context); + parameter_values[identifier->name()] = convertFieldToString(res->as()->value); + } } } } }; -NameToNameMap analyzeFunctionParamValues(const ASTPtr & ast) +NameToNameMap analyzeFunctionParamValues(const ASTPtr & ast, ContextPtr context) { NameToNameMap parameter_values; - FunctionParameterValuesVisitor(parameter_values).visit(ast); + FunctionParameterValuesVisitor(parameter_values, context).visit(ast); return parameter_values; } diff --git a/src/Parsers/FunctionParameterValuesVisitor.h b/src/Parsers/FunctionParameterValuesVisitor.h index e6ce0e42d06..8c2686dcc65 100644 --- a/src/Parsers/FunctionParameterValuesVisitor.h +++ b/src/Parsers/FunctionParameterValuesVisitor.h @@ -2,12 +2,13 @@ #include #include +#include namespace DB { /// Find parameters in a query parameter values and collect them into map. -NameToNameMap analyzeFunctionParamValues(const ASTPtr & ast); +NameToNameMap analyzeFunctionParamValues(const ASTPtr & ast, ContextPtr context); } diff --git a/tests/queries/0_stateless/03146_parameterized_view_with_date.reference b/tests/queries/0_stateless/03146_parameterized_view_with_date.reference new file mode 100644 index 00000000000..1d6227dbbcb --- /dev/null +++ b/tests/queries/0_stateless/03146_parameterized_view_with_date.reference @@ -0,0 +1 @@ +2 2024-04-01 01:00:00 diff --git a/tests/queries/0_stateless/03146_parameterized_view_with_date.sql b/tests/queries/0_stateless/03146_parameterized_view_with_date.sql new file mode 100644 index 00000000000..53022e969ab --- /dev/null +++ b/tests/queries/0_stateless/03146_parameterized_view_with_date.sql @@ -0,0 +1,12 @@ + +drop table if exists table_pv; +create table table_pv (id Int32, timestamp_field DateTime) engine = Memory(); + +insert into table_pv values(1, '2024-03-01 00:00:00'); +insert into table_pv values (2, '2024-04-01 01:00:00'); + +create view pv as select * from table_pv where timestamp_field > {timestamp_param:DateTime}; + +select * from pv (timestamp_param=toDateTime('2024-04-01 00:00:01')); + +drop table table_pv; From e2b7ca7d1137aed5c449eb9750b97e9be567681b Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 29 May 2024 11:54:31 -0400 Subject: [PATCH 02/43] Fix restore from backup for definers --- src/Interpreters/InterpreterCreateQuery.cpp | 8 +++---- src/Interpreters/InterpreterCreateQuery.h | 2 +- .../test_backup_restore_new/test.py | 24 +++++++++++++++++++ 3 files changed, 29 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index b30fc8bc092..bf6c4f68947 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1086,7 +1086,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) create.sql_security = std::make_shared(); if (create.sql_security) - processSQLSecurityOption(getContext(), create.sql_security->as(), create.attach, create.is_materialized_view); + processSQLSecurityOption(getContext(), create.sql_security->as(), create.attach, create.is_materialized_view, is_restore_from_backup); DDLGuardPtr ddl_guard; @@ -1880,7 +1880,7 @@ void InterpreterCreateQuery::addColumnsDescriptionToCreateQueryIfNecessary(ASTCr } } -void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach, bool is_materialized_view) +void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach, bool is_materialized_view, bool is_restore_from_backup_) { /// If no SQL security is specified, apply default from default_*_view_sql_security setting. if (!sql_security.type) @@ -1921,7 +1921,7 @@ void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQ } /// Checks the permissions for the specified definer user. - if (sql_security.definer && !sql_security.is_definer_current_user && !is_attach) + if (sql_security.definer && !sql_security.is_definer_current_user && !is_attach && !is_restore_from_backup_) { const auto definer_name = sql_security.definer->toString(); @@ -1931,7 +1931,7 @@ void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQ context_->checkAccess(AccessType::SET_DEFINER, definer_name); } - if (sql_security.type == SQLSecurityType::NONE && !is_attach) + if (sql_security.type == SQLSecurityType::NONE && !is_attach && !is_restore_from_backup_) context_->checkAccess(AccessType::ALLOW_SQL_SECURITY_NONE); } diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index be4a10eaf1d..c4bfa7b36d8 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -82,7 +82,7 @@ public: void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const override; /// Check access right, validate definer statement and replace `CURRENT USER` with actual name. - static void processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach = false, bool is_materialized_view = false); + static void processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach = false, bool is_materialized_view = false, bool is_restore_from_backup_ = false); private: struct TableProperties diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index ef9e536976b..a6621dce231 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -168,6 +168,30 @@ def test_restore_table(engine): assert instance.query("SELECT count(), sum(x) FROM test.table") == "100\t4950\n" +def test_restore_materialized_view_with_definer(): + instance.query("CREATE DATABASE test") + instance.query("CREATE TABLE test.test_table (s String) ENGINE = MergeTree ORDER BY s") + instance.query("CREATE USER u1") + instance.query("GRANT SELECT ON *.* TO u1") + instance.query("GRANT INSERT ON *.* TO u1") + + instance.query( + """ + CREATE MATERIALIZED VIEW test.test_mv_1 (s String) + ENGINE = MergeTree ORDER BY s + DEFINER = u1 SQL SECURITY DEFINER + AS SELECT * FROM test.test_table + """ + ) + + backup_name = new_backup_name() + instance.query(f"BACKUP DATABASE test TO {backup_name}") + instance.query("DROP DATABASE test") + instance.query("DROP USER u1") + + instance.query(f"RESTORE DATABASE test FROM {backup_name}") + + @pytest.mark.parametrize( "engine", ["MergeTree", "Log", "TinyLog", "StripeLog", "Memory"] ) From b908421259dab2849c5da8c01d8909ba8966e6c1 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 29 May 2024 16:05:50 +0000 Subject: [PATCH 03/43] Automatic style fix --- tests/integration/test_backup_restore_new/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_backup_restore_new/test.py b/tests/integration/test_backup_restore_new/test.py index a6621dce231..f5a63cd69a3 100644 --- a/tests/integration/test_backup_restore_new/test.py +++ b/tests/integration/test_backup_restore_new/test.py @@ -170,7 +170,9 @@ def test_restore_table(engine): def test_restore_materialized_view_with_definer(): instance.query("CREATE DATABASE test") - instance.query("CREATE TABLE test.test_table (s String) ENGINE = MergeTree ORDER BY s") + instance.query( + "CREATE TABLE test.test_table (s String) ENGINE = MergeTree ORDER BY s" + ) instance.query("CREATE USER u1") instance.query("GRANT SELECT ON *.* TO u1") instance.query("GRANT INSERT ON *.* TO u1") From f69918dba0f0dcd3bd90f58691635423d5c3c0b5 Mon Sep 17 00:00:00 2001 From: pufit Date: Wed, 29 May 2024 16:03:29 -0400 Subject: [PATCH 04/43] skip_check_permissions --- src/Interpreters/InterpreterCreateQuery.cpp | 8 ++++---- src/Interpreters/InterpreterCreateQuery.h | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index bf6c4f68947..a3eaced0d02 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1086,7 +1086,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) create.sql_security = std::make_shared(); if (create.sql_security) - processSQLSecurityOption(getContext(), create.sql_security->as(), create.attach, create.is_materialized_view, is_restore_from_backup); + processSQLSecurityOption(getContext(), create.sql_security->as(), create.is_materialized_view, /* skip_check_permissions= */ is_restore_from_backup || create.attach); DDLGuardPtr ddl_guard; @@ -1880,7 +1880,7 @@ void InterpreterCreateQuery::addColumnsDescriptionToCreateQueryIfNecessary(ASTCr } } -void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach, bool is_materialized_view, bool is_restore_from_backup_) +void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_materialized_view, bool skip_check_permissions) { /// If no SQL security is specified, apply default from default_*_view_sql_security setting. if (!sql_security.type) @@ -1921,7 +1921,7 @@ void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQ } /// Checks the permissions for the specified definer user. - if (sql_security.definer && !sql_security.is_definer_current_user && !is_attach && !is_restore_from_backup_) + if (sql_security.definer && !sql_security.is_definer_current_user && !skip_check_permissions) { const auto definer_name = sql_security.definer->toString(); @@ -1931,7 +1931,7 @@ void InterpreterCreateQuery::processSQLSecurityOption(ContextPtr context_, ASTSQ context_->checkAccess(AccessType::SET_DEFINER, definer_name); } - if (sql_security.type == SQLSecurityType::NONE && !is_attach && !is_restore_from_backup_) + if (sql_security.type == SQLSecurityType::NONE && !skip_check_permissions) context_->checkAccess(AccessType::ALLOW_SQL_SECURITY_NONE); } diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index c4bfa7b36d8..70ef29e6b07 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -82,7 +82,7 @@ public: void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const override; /// Check access right, validate definer statement and replace `CURRENT USER` with actual name. - static void processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_attach = false, bool is_materialized_view = false, bool is_restore_from_backup_ = false); + static void processSQLSecurityOption(ContextPtr context_, ASTSQLSecurity & sql_security, bool is_materialized_view = false, bool skip_check_permissions = false); private: struct TableProperties From 63900550c9adba47034f0f136c45d4ef4d34076f Mon Sep 17 00:00:00 2001 From: pufit Date: Mon, 3 Jun 2024 19:16:42 -0400 Subject: [PATCH 05/43] Making use of LoadingStrictnessLevel --- src/Interpreters/InterpreterCreateQuery.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a3eaced0d02..5e07f7a27f2 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1082,11 +1082,14 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) String current_database = getContext()->getCurrentDatabase(); auto database_name = create.database ? create.getDatabase() : current_database; + bool is_secondary_query = getContext()->getZooKeeperMetadataTransaction() && !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery(); + auto mode = getLoadingStrictnessLevel(create.attach, /*force_attach*/ false, /*has_force_restore_data_flag*/ false, is_secondary_query || is_restore_from_backup); + if (!create.sql_security && create.supportSQLSecurity() && !getContext()->getServerSettings().ignore_empty_sql_security_in_create_view_query) create.sql_security = std::make_shared(); if (create.sql_security) - processSQLSecurityOption(getContext(), create.sql_security->as(), create.is_materialized_view, /* skip_check_permissions= */ is_restore_from_backup || create.attach); + processSQLSecurityOption(getContext(), create.sql_security->as(), create.is_materialized_view, /* skip_check_permissions= */ mode >= LoadingStrictnessLevel::SECONDARY_CREATE); DDLGuardPtr ddl_guard; @@ -1213,9 +1216,6 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!UserDefinedSQLFunctionFactory::instance().empty()) UserDefinedSQLFunctionVisitor::visit(query_ptr); - bool is_secondary_query = getContext()->getZooKeeperMetadataTransaction() && !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery(); - auto mode = getLoadingStrictnessLevel(create.attach, /*force_attach*/ false, /*has_force_restore_data_flag*/ false, is_secondary_query || is_restore_from_backup); - /// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way. TableProperties properties = getTablePropertiesAndNormalizeCreateQuery(create, mode); From 823a7d37f3bab8d4aa9cb8b2ff8b7b087e8a3037 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 6 Jun 2024 20:18:42 +0200 Subject: [PATCH 06/43] support statistics on replicated merge tree --- src/Interpreters/InterpreterCreateQuery.cpp | 6 ++--- src/Parsers/ASTColumnDeclaration.cpp | 10 +++---- src/Parsers/ASTColumnDeclaration.h | 2 +- src/Parsers/ParserCreateQuery.h | 10 +++---- src/Storages/ColumnsDescription.cpp | 4 +++ src/Storages/StatisticsDescription.cpp | 2 +- .../test_manipulate_statistics/test.py | 27 ++++++++++++++++++- 7 files changed, 45 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 66936dc25d7..a51d3e6dade 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -450,8 +450,8 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) if (!column.statistics.empty()) { - column_declaration->stat_type = column.statistics.getAST(); - column_declaration->children.push_back(column_declaration->stat_type); + column_declaration->statistics_desc = column.statistics.getAST(); + column_declaration->children.push_back(column_declaration->statistics_desc); } if (column.ttl) @@ -676,7 +676,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( } column.statistics.column_name = column.name; /// We assign column name here for better exception error message. - if (col_decl.stat_type) + if (col_decl.statistics_desc) { if (!skip_checks && !context_->getSettingsRef().allow_experimental_statistics) throw Exception(ErrorCodes::INCORRECT_QUERY, "Create table with statistics is now disabled. Turn on allow_experimental_statistics"); diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 6c29e0bf9d5..4a8a3d2967d 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -39,10 +39,10 @@ ASTPtr ASTColumnDeclaration::clone() const res->children.push_back(res->codec); } - if (stat_type) + if (statistics_desc) { - res->stat_type = stat_type->clone(); - res->children.push_back(res->stat_type); + res->statistics_desc = statistics_desc->clone(); + res->children.push_back(res->statistics_desc); } if (ttl) @@ -111,10 +111,10 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo codec->formatImpl(format_settings, state, frame); } - if (stat_type) + if (statistics_desc) { format_settings.ostr << ' '; - stat_type->formatImpl(format_settings, state, frame); + statistics_desc->formatImpl(format_settings, state, frame); } if (ttl) diff --git a/src/Parsers/ASTColumnDeclaration.h b/src/Parsers/ASTColumnDeclaration.h index d775928d05c..914916d5074 100644 --- a/src/Parsers/ASTColumnDeclaration.h +++ b/src/Parsers/ASTColumnDeclaration.h @@ -19,7 +19,7 @@ public: bool ephemeral_default = false; ASTPtr comment; ASTPtr codec; - ASTPtr stat_type; + ASTPtr statistics_desc; ASTPtr ttl; ASTPtr collation; ASTPtr settings; diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 27bb524970d..5f6df33176f 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -193,7 +193,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ASTPtr default_expression; ASTPtr comment_expression; ASTPtr codec_expression; - ASTPtr stat_type_expression; + ASTPtr statistics_desc_expression; ASTPtr ttl_expression; ASTPtr collation_expression; ASTPtr settings; @@ -325,7 +325,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E if (s_stat.ignore(pos, expected)) { - if (!stat_type_parser.parse(pos, stat_type_expression, expected)) + if (!stat_type_parser.parse(pos, statistics_desc_expression, expected)) return false; } @@ -398,10 +398,10 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E column_declaration->children.push_back(std::move(settings)); } - if (stat_type_expression) + if (statistics_desc_expression) { - column_declaration->stat_type = stat_type_expression; - column_declaration->children.push_back(std::move(stat_type_expression)); + column_declaration->statistics_desc = statistics_desc_expression; + column_declaration->children.push_back(std::move(statistics_desc_expression)); } if (ttl_expression) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 69e39323219..d1babb817bf 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -25,6 +25,7 @@ #include #include #include "Parsers/ASTSetQuery.h" +#include "Storages/StatisticsDescription.h" #include #include #include @@ -207,6 +208,9 @@ void ColumnDescription::readText(ReadBuffer & buf) if (col_ast->settings) settings = col_ast->settings->as().changes; + + if (col_ast->statistics_desc) + statistics = ColumnStatisticsDescription::fromColumnDeclaration(*col_ast); } else throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description"); diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index dff1b7d3602..fc06c66b50e 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -171,7 +171,7 @@ std::vector ColumnStatisticsDescription::fromAST(co ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(const ASTColumnDeclaration & column) { - const auto & stat_type_list_ast = column.stat_type->as().arguments; + const auto & stat_type_list_ast = column.statistics_desc->as().arguments; if (stat_type_list_ast->children.empty()) throw Exception(ErrorCodes::INCORRECT_QUERY, "We expect at least one statistics type for column {}", queryToString(column)); ColumnStatisticsDescription stats; diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index 2b26af940d1..bffee89ffc6 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -6,9 +6,12 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance( - "node1", user_configs=["config/config.xml"], with_zookeeper=False + "node1", user_configs=["config/config.xml"], with_zookeeper=True ) +node2 = cluster.add_instance( + "node2", user_configs=["config/config.xml"], with_zookeeper=True +) @pytest.fixture(scope="module") def started_cluster(): @@ -122,3 +125,25 @@ def test_single_node_normal(started_cluster): """ ) run_test_single_node(started_cluster) + +def test_replicated_table_ddl(started_cluster): + node1.query("DROP TABLE IF EXISTS test_stat") + node2.query("DROP TABLE IF EXISTS test_stat") + + node1.query( + """ + CREATE TABLE test_stat(a Int64 STATISTICS(tdigest, uniq), b Int64 STATISTICS(tdigest, uniq), c Int64 STATISTICS(tdigest)) + ENGINE = ReplicatedMergeTree('/clickhouse/test/statistics', '1') ORDER BY a; + """ + ) + node2.query( + """ + CREATE TABLE test_stat(a Int64 STATISTICS(tdigest, uniq), b Int64 STATISTICS(tdigest, uniq), c Int64 STATISTICS(tdigest)) + ENGINE = ReplicatedMergeTree('/clickhouse/test/statistics', '2') ORDER BY a; + """ + ) + + node1.query("ALTER TABLE test_stat MODIFY STATISTICS c TYPE tdigest, uniq", settings={"alter_sync":"2"}); + node1.query("ALTER TABLE test_stat DROP STATISTICS b", settings={"alter_sync":"2"}); + + assert node2.query("SHOW CREATE TABLE test_stat") == "CREATE TABLE default.test_stat\\n(\\n `a` Int64 STATISTICS(tdigest, uniq),\\n `b` Int64,\\n `c` Int64 STATISTICS(tdigest, uniq)\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/test/statistics\\', \\'2\\')\\nORDER BY a\\nSETTINGS index_granularity = 8192\n" From 0bd5164f83b1751999c667de4ec6327f9a59c5d6 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 6 Jun 2024 21:30:16 +0200 Subject: [PATCH 07/43] fix style --- src/Storages/ColumnsDescription.cpp | 3 +-- .../integration/test_manipulate_statistics/test.py | 14 +++++++++++--- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index d1babb817bf..556f8a6e42d 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -24,8 +25,6 @@ #include #include #include -#include "Parsers/ASTSetQuery.h" -#include "Storages/StatisticsDescription.h" #include #include #include diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index bffee89ffc6..53ab9682ad2 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -13,6 +13,7 @@ node2 = cluster.add_instance( "node2", user_configs=["config/config.xml"], with_zookeeper=True ) + @pytest.fixture(scope="module") def started_cluster(): try: @@ -126,6 +127,7 @@ def test_single_node_normal(started_cluster): ) run_test_single_node(started_cluster) + def test_replicated_table_ddl(started_cluster): node1.query("DROP TABLE IF EXISTS test_stat") node2.query("DROP TABLE IF EXISTS test_stat") @@ -143,7 +145,13 @@ def test_replicated_table_ddl(started_cluster): """ ) - node1.query("ALTER TABLE test_stat MODIFY STATISTICS c TYPE tdigest, uniq", settings={"alter_sync":"2"}); - node1.query("ALTER TABLE test_stat DROP STATISTICS b", settings={"alter_sync":"2"}); + node1.query( + "ALTER TABLE test_stat MODIFY STATISTICS c TYPE tdigest, uniq", + settings={"alter_sync": "2"}, + ) + node1.query("ALTER TABLE test_stat DROP STATISTICS b", settings={"alter_sync": "2"}) - assert node2.query("SHOW CREATE TABLE test_stat") == "CREATE TABLE default.test_stat\\n(\\n `a` Int64 STATISTICS(tdigest, uniq),\\n `b` Int64,\\n `c` Int64 STATISTICS(tdigest, uniq)\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/test/statistics\\', \\'2\\')\\nORDER BY a\\nSETTINGS index_granularity = 8192\n" + assert ( + node2.query("SHOW CREATE TABLE test_stat") + == "CREATE TABLE default.test_stat\\n(\\n `a` Int64 STATISTICS(tdigest, uniq),\\n `b` Int64,\\n `c` Int64 STATISTICS(tdigest, uniq)\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/test/statistics\\', \\'2\\')\\nORDER BY a\\nSETTINGS index_granularity = 8192\n" + ) From de5258128e48061514a82bdfdc5368852d6a5062 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 7 Jun 2024 06:44:36 +0000 Subject: [PATCH 08/43] update fmtlib version to 9.1.0 Signed-off-by: Duc Canh Le --- base/base/EnumReflection.h | 2 +- base/base/wide_integer_to_string.h | 2 +- contrib/fmtlib | 2 +- contrib/fmtlib-cmake/CMakeLists.txt | 1 - src/Analyzer/Identifier.h | 4 ++-- src/Client/TestHint.h | 2 +- src/Common/TransactionID.h | 2 +- src/Common/ZooKeeper/IKeeper.h | 2 +- src/Common/filesystemHelpers.cpp | 13 ++++++++----- src/Common/formatReadable.h | 2 +- src/Coordination/KeeperSnapshotManager.cpp | 2 +- src/Coordination/RaftServerConfig.h | 4 ++-- src/Core/Field.h | 2 +- src/Core/QualifiedTableName.h | 2 +- src/DataTypes/IDataType.h | 2 +- src/Functions/FunctionsStringDistance.cpp | 4 ++-- src/IO/WriteHelpers.h | 2 +- src/Interpreters/StorageID.h | 2 +- src/Parsers/formatAST.h | 2 +- .../Formats/Impl/BSONEachRowRowInputFormat.cpp | 4 ++-- src/Processors/Transforms/WindowTransform.cpp | 2 +- src/Storages/Kafka/KafkaConsumer.cpp | 5 +---- src/Storages/Kafka/KafkaConsumer.h | 5 +++++ src/Storages/MergeTree/MarkRange.h | 2 +- .../ParallelReplicasReadingCoordinator.cpp | 2 +- src/Storages/MergeTree/RangesInDataPart.cpp | 2 +- src/Storages/NamedCollectionsHelpers.h | 2 +- 27 files changed, 41 insertions(+), 37 deletions(-) diff --git a/base/base/EnumReflection.h b/base/base/EnumReflection.h index 4a9de4d17a3..e4e0ef672fd 100644 --- a/base/base/EnumReflection.h +++ b/base/base/EnumReflection.h @@ -32,7 +32,7 @@ constexpr void static_for(F && f) template struct fmt::formatter : fmt::formatter { - constexpr auto format(T value, auto& format_context) + constexpr auto format(T value, auto& format_context) const { return formatter::format(magic_enum::enum_name(value), format_context); } diff --git a/base/base/wide_integer_to_string.h b/base/base/wide_integer_to_string.h index c2cbe8d82e3..f703a722afa 100644 --- a/base/base/wide_integer_to_string.h +++ b/base/base/wide_integer_to_string.h @@ -62,7 +62,7 @@ struct fmt::formatter> } template - auto format(const wide::integer & value, FormatContext & ctx) + auto format(const wide::integer & value, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", to_string(value)); } diff --git a/contrib/fmtlib b/contrib/fmtlib index b6f4ceaed0a..a33701196ad 160000 --- a/contrib/fmtlib +++ b/contrib/fmtlib @@ -1 +1 @@ -Subproject commit b6f4ceaed0a0a24ccf575fab6c56dd50ccf6f1a9 +Subproject commit a33701196adfad74917046096bf5a2aa0ab0bb50 diff --git a/contrib/fmtlib-cmake/CMakeLists.txt b/contrib/fmtlib-cmake/CMakeLists.txt index fe399ddc6e1..6625e411295 100644 --- a/contrib/fmtlib-cmake/CMakeLists.txt +++ b/contrib/fmtlib-cmake/CMakeLists.txt @@ -13,7 +13,6 @@ set (SRCS ${FMT_SOURCE_DIR}/include/fmt/core.h ${FMT_SOURCE_DIR}/include/fmt/format.h ${FMT_SOURCE_DIR}/include/fmt/format-inl.h - ${FMT_SOURCE_DIR}/include/fmt/locale.h ${FMT_SOURCE_DIR}/include/fmt/os.h ${FMT_SOURCE_DIR}/include/fmt/ostream.h ${FMT_SOURCE_DIR}/include/fmt/printf.h diff --git a/src/Analyzer/Identifier.h b/src/Analyzer/Identifier.h index cbd8f5e7694..91190dc7cdb 100644 --- a/src/Analyzer/Identifier.h +++ b/src/Analyzer/Identifier.h @@ -406,7 +406,7 @@ struct fmt::formatter } template - auto format(const DB::Identifier & identifier, FormatContext & ctx) + auto format(const DB::Identifier & identifier, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", identifier.getFullName()); } @@ -428,7 +428,7 @@ struct fmt::formatter } template - auto format(const DB::IdentifierView & identifier_view, FormatContext & ctx) + auto format(const DB::IdentifierView & identifier_view, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", identifier_view.getFullName()); } diff --git a/src/Client/TestHint.h b/src/Client/TestHint.h index eaf854be5df..b76c4245df4 100644 --- a/src/Client/TestHint.h +++ b/src/Client/TestHint.h @@ -112,7 +112,7 @@ struct fmt::formatter } template - auto format(const DB::TestHint::ErrorVector & ErrorVector, FormatContext & ctx) + auto format(const DB::TestHint::ErrorVector & ErrorVector, FormatContext & ctx) const { if (ErrorVector.empty()) return fmt::format_to(ctx.out(), "{}", 0); diff --git a/src/Common/TransactionID.h b/src/Common/TransactionID.h index 97d0072bc14..466f3f5343b 100644 --- a/src/Common/TransactionID.h +++ b/src/Common/TransactionID.h @@ -108,7 +108,7 @@ struct fmt::formatter } template - auto format(const DB::TransactionID & tid, FormatContext & context) + auto format(const DB::TransactionID & tid, FormatContext & context) const { return fmt::format_to(context.out(), "({}, {}, {})", tid.start_csn, tid.local_tid, tid.host_id); } diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index ddd30c4eef2..7d574247aa5 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -647,7 +647,7 @@ public: template <> struct fmt::formatter : fmt::formatter { - constexpr auto format(Coordination::Error code, auto & ctx) + constexpr auto format(Coordination::Error code, auto & ctx) const { return formatter::format(Coordination::errorMessage(code), ctx); } diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 2d053c615d9..f8d209bc11f 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -1,4 +1,6 @@ #include "filesystemHelpers.h" +#include +#include #if defined(OS_LINUX) # include @@ -11,7 +13,7 @@ #include #include #include -#include +#include #include #include #include @@ -369,10 +371,11 @@ Poco::Timestamp getModificationTimestamp(const std::string & path) void setModificationTime(const std::string & path, time_t time) { - struct utimbuf tb; - tb.actime = time; - tb.modtime = time; - if (utime(path.c_str(), &tb) != 0) + struct timeval times[2]; + times[0].tv_usec = times[1].tv_usec = 0; + times[0].tv_sec = ::time(nullptr); + times[1].tv_sec = time; + if (utimes(path.c_str(), times) != 0) DB::ErrnoException::throwFromPath(DB::ErrorCodes::PATH_ACCESS_DENIED, path, "Cannot set modification time to file: {}", path); } diff --git a/src/Common/formatReadable.h b/src/Common/formatReadable.h index a05a2a7f9e2..0d7a437219a 100644 --- a/src/Common/formatReadable.h +++ b/src/Common/formatReadable.h @@ -49,7 +49,7 @@ struct fmt::formatter } template - auto format(const ReadableSize & size, FormatContext & ctx) + auto format(const ReadableSize & size, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", formatReadableSizeWithBinarySuffix(size.value)); } diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index f25ccab86b1..23ff714a929 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -466,7 +466,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial #ifdef NDEBUG /// TODO (alesapin) remove this, it should be always CORRUPTED_DATA. LOG_ERROR(getLogger("KeeperSnapshotManager"), "Children counter in stat.numChildren {}" - " is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key); + " is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key.toView()); #else throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}" " is different from actual children size {} for node {}", diff --git a/src/Coordination/RaftServerConfig.h b/src/Coordination/RaftServerConfig.h index 0ecbd6464c1..37b6a92ba70 100644 --- a/src/Coordination/RaftServerConfig.h +++ b/src/Coordination/RaftServerConfig.h @@ -57,7 +57,7 @@ using ClusterUpdateActions = std::vector; template <> struct fmt::formatter : fmt::formatter { - constexpr auto format(const DB::RaftServerConfig & server, format_context & ctx) + constexpr auto format(const DB::RaftServerConfig & server, format_context & ctx) const { return fmt::format_to( ctx.out(), "server.{}={};{};{}", server.id, server.endpoint, server.learner ? "learner" : "participant", server.priority); @@ -67,7 +67,7 @@ struct fmt::formatter : fmt::formatter template <> struct fmt::formatter : fmt::formatter { - constexpr auto format(const DB::ClusterUpdateAction & action, format_context & ctx) + constexpr auto format(const DB::ClusterUpdateAction & action, format_context & ctx) const { if (const auto * add = std::get_if(&action)) return fmt::format_to(ctx.out(), "(Add server {})", add->id); diff --git a/src/Core/Field.h b/src/Core/Field.h index a78b589c883..f1bb4a72b0d 100644 --- a/src/Core/Field.h +++ b/src/Core/Field.h @@ -1038,7 +1038,7 @@ struct fmt::formatter } template - auto format(const DB::Field & x, FormatContext & ctx) + auto format(const DB::Field & x, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", toString(x)); } diff --git a/src/Core/QualifiedTableName.h b/src/Core/QualifiedTableName.h index bf05bd59caf..0fd72c32a54 100644 --- a/src/Core/QualifiedTableName.h +++ b/src/Core/QualifiedTableName.h @@ -125,7 +125,7 @@ namespace fmt } template - auto format(const DB::QualifiedTableName & name, FormatContext & ctx) + auto format(const DB::QualifiedTableName & name, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}.{}", DB::backQuoteIfNeed(name.database), DB::backQuoteIfNeed(name.table)); } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 46c30240ef8..bd6065ca270 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -623,7 +623,7 @@ struct fmt::formatter } template - auto format(const DB::DataTypePtr & type, FormatContext & ctx) + auto format(const DB::DataTypePtr & type, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", type->getName()); } diff --git a/src/Functions/FunctionsStringDistance.cpp b/src/Functions/FunctionsStringDistance.cpp index 6cb23bbea9f..d0d8ebc946d 100644 --- a/src/Functions/FunctionsStringDistance.cpp +++ b/src/Functions/FunctionsStringDistance.cpp @@ -159,7 +159,7 @@ struct ByteJaccardIndexImpl } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", StringRef(haystack, haystack_end - haystack)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", std::string_view(haystack, haystack_end - haystack)); } } } @@ -186,7 +186,7 @@ struct ByteJaccardIndexImpl } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", StringRef(needle, needle_end - needle)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", std::string_view(needle, needle_end - needle)); } } } diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index d4b2d8ea0dc..6b0de441e94 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1420,7 +1420,7 @@ struct fmt::formatter } template - auto format(const DB::UUID & uuid, FormatContext & context) + auto format(const DB::UUID & uuid, FormatContext & context) const { return fmt::format_to(context.out(), "{}", toString(uuid)); } diff --git a/src/Interpreters/StorageID.h b/src/Interpreters/StorageID.h index 96e3cefe00c..69dac8ea32d 100644 --- a/src/Interpreters/StorageID.h +++ b/src/Interpreters/StorageID.h @@ -136,7 +136,7 @@ namespace fmt } template - auto format(const DB::StorageID & storage_id, FormatContext & ctx) + auto format(const DB::StorageID & storage_id, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", storage_id.getNameForLogs()); } diff --git a/src/Parsers/formatAST.h b/src/Parsers/formatAST.h index dd72a59b4a2..e34902663dd 100644 --- a/src/Parsers/formatAST.h +++ b/src/Parsers/formatAST.h @@ -40,7 +40,7 @@ struct fmt::formatter } template - auto format(const DB::ASTPtr & ast, FormatContext & context) + auto format(const DB::ASTPtr & ast, FormatContext & context) const { return fmt::format_to(context.out(), "{}", DB::serializeAST(*ast)); } diff --git a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp index 6a3475a1830..56c774782c2 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp @@ -421,7 +421,7 @@ void BSONEachRowRowInputFormat::readTuple(IColumn & column, const DataTypePtr & "Cannot parse tuple column with type {} from BSON array/embedded document field: " "tuple doesn't have element with name \"{}\"", data_type->getName(), - name); + name.toView()); index = *try_get_index; } @@ -806,7 +806,7 @@ bool BSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi else { if (seen_columns[index]) - throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing BSONEachRow format: {}", name); + throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing BSONEachRow format: {}", name.toView()); seen_columns[index] = true; read_columns[index] = readField(*columns[index], types[index], BSONType(type)); diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index af340c4aab8..b9f61d30182 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -37,7 +37,7 @@ struct fmt::formatter } template - auto format(const DB::RowNumber & x, FormatContext & ctx) + auto format(const DB::RowNumber & x, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}:{}", x.block, x.row); } diff --git a/src/Storages/Kafka/KafkaConsumer.cpp b/src/Storages/Kafka/KafkaConsumer.cpp index 7075dcb71ca..9ba42b9875e 100644 --- a/src/Storages/Kafka/KafkaConsumer.cpp +++ b/src/Storages/Kafka/KafkaConsumer.cpp @@ -1,7 +1,4 @@ -// Needs to go first because its partial specialization of fmt::formatter -// should be defined before any instantiation -#include - +#include #include #include diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index a3bc97779b3..a2d047933be 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -1,5 +1,7 @@ #pragma once +#include +#include #include #include @@ -197,3 +199,6 @@ private: }; } + +template <> struct fmt::formatter : fmt::ostream_formatter {}; +template <> struct fmt::formatter : fmt::ostream_formatter {}; diff --git a/src/Storages/MergeTree/MarkRange.h b/src/Storages/MergeTree/MarkRange.h index 626d4e9e689..6b111f348bb 100644 --- a/src/Storages/MergeTree/MarkRange.h +++ b/src/Storages/MergeTree/MarkRange.h @@ -69,7 +69,7 @@ struct fmt::formatter } template - auto format(const DB::MarkRange & range, FormatContext & ctx) + auto format(const DB::MarkRange & range, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", fmt::format("({}, {})", range.begin, range.end)); } diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index f3318a48883..79c0e6ad262 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -112,7 +112,7 @@ struct fmt::formatter static constexpr auto parse(format_parse_context & ctx) { return ctx.begin(); } template - auto format(const DB::Part & part, FormatContext & ctx) + auto format(const DB::Part & part, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{} in replicas [{}]", part.description.describe(), fmt::join(part.replicas, ", ")); } diff --git a/src/Storages/MergeTree/RangesInDataPart.cpp b/src/Storages/MergeTree/RangesInDataPart.cpp index c46385e84ef..50e0781b4e6 100644 --- a/src/Storages/MergeTree/RangesInDataPart.cpp +++ b/src/Storages/MergeTree/RangesInDataPart.cpp @@ -13,7 +13,7 @@ struct fmt::formatter static constexpr auto parse(format_parse_context & ctx) { return ctx.begin(); } template - auto format(const DB::RangesInDataPartDescription & range, FormatContext & ctx) + auto format(const DB::RangesInDataPartDescription & range, FormatContext & ctx) const { return fmt::format_to(ctx.out(), "{}", range.describe()); } diff --git a/src/Storages/NamedCollectionsHelpers.h b/src/Storages/NamedCollectionsHelpers.h index a1909f514ea..b4aea096c59 100644 --- a/src/Storages/NamedCollectionsHelpers.h +++ b/src/Storages/NamedCollectionsHelpers.h @@ -158,7 +158,7 @@ struct fmt::formatter> } template - auto format(const DB::NamedCollectionValidateKey & elem, FormatContext & context) + auto format(const DB::NamedCollectionValidateKey & elem, FormatContext & context) const { return fmt::format_to(context.out(), "{}", elem.value); } From e6f88126964f3018f07d4137e89541f8b45806a3 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 10 Jun 2024 06:53:25 +0000 Subject: [PATCH 09/43] fix mistake Signed-off-by: Duc Canh Le --- src/Common/filesystemHelpers.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index f8d209bc11f..09c4508b7b2 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -1,5 +1,5 @@ #include "filesystemHelpers.h" -#include +#include #include #if defined(OS_LINUX) @@ -371,11 +371,10 @@ Poco::Timestamp getModificationTimestamp(const std::string & path) void setModificationTime(const std::string & path, time_t time) { - struct timeval times[2]; - times[0].tv_usec = times[1].tv_usec = 0; - times[0].tv_sec = ::time(nullptr); - times[1].tv_sec = time; - if (utimes(path.c_str(), times) != 0) + struct utimbuf tb; + tb.actime = time; + tb.modtime = time; + if (utime(path.c_str(), &tb) != 0) DB::ErrnoException::throwFromPath(DB::ErrorCodes::PATH_ACCESS_DENIED, path, "Cannot set modification time to file: {}", path); } From a50019c6ca2f4a20b59866fac68387a63787ab0e Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 10 Jun 2024 11:49:41 +0000 Subject: [PATCH 10/43] Add metrics for PutObject and GetObject throttlers of ObjectStorage --- src/Common/Throttler.cpp | 39 ++++++++++++------- src/Common/Throttler.h | 6 +++ src/IO/S3/Client.h | 3 ++ .../ServerAsynchronousMetrics.cpp | 25 ++++++++++++ 4 files changed, 60 insertions(+), 13 deletions(-) diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index 4c1320db27a..61d120e11ea 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -43,19 +43,7 @@ UInt64 Throttler::add(size_t amount) // Values obtained under lock to be checked after release size_t count_value; double tokens_value; - { - std::lock_guard lock(mutex); - auto now = clock_gettime_ns_adjusted(prev_ns); - if (max_speed) - { - double delta_seconds = prev_ns ? static_cast(now - prev_ns) / NS : 0; - tokens = std::min(tokens + max_speed * delta_seconds - amount, max_burst); - } - count += amount; - count_value = count; - tokens_value = tokens; - prev_ns = now; - } + addImpl(amount, count_value, tokens_value); if (limit && count_value > limit) throw Exception::createDeprecated(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED); @@ -77,6 +65,21 @@ UInt64 Throttler::add(size_t amount) return static_cast(sleep_time_ns); } +void Throttler::addImpl(size_t amount, size_t & count_value, double & tokens_value) +{ + std::lock_guard lock(mutex); + auto now = clock_gettime_ns_adjusted(prev_ns); + if (max_speed) + { + double delta_seconds = prev_ns ? static_cast(now - prev_ns) / NS : 0; + tokens = std::min(tokens + max_speed * delta_seconds - amount, max_burst); + } + count += amount; + count_value = count; + tokens_value = tokens; + prev_ns = now; +} + void Throttler::reset() { std::lock_guard lock(mutex); @@ -98,4 +101,14 @@ bool Throttler::isThrottling() const return false; } +Int64 Throttler::getAvailable() +{ + // To update bucket state and receive current number of token in a thread-safe way + size_t count_value; + double tokens_value; + addImpl(0, count_value, tokens_value); + + return static_cast(tokens_value); +} + } diff --git a/src/Common/Throttler.h b/src/Common/Throttler.h index 7508065096b..32293d7400f 100644 --- a/src/Common/Throttler.h +++ b/src/Common/Throttler.h @@ -57,7 +57,13 @@ public: /// Is throttler already accumulated some sleep time and throttling. bool isThrottling() const; + Int64 getAvailable(); + UInt64 getMaxSpeed() const { return static_cast(max_speed); } + UInt64 getMaxBurst() const { return static_cast(max_burst); } + private: + void addImpl(size_t amount, size_t & count_value, double & tokens_value); + size_t count{0}; const size_t max_speed{0}; /// in tokens per second. const size_t max_burst{0}; /// in tokens. diff --git a/src/IO/S3/Client.h b/src/IO/S3/Client.h index bd281846343..2fd0a9cd2d1 100644 --- a/src/IO/S3/Client.h +++ b/src/IO/S3/Client.h @@ -219,6 +219,9 @@ public: return client_configuration.for_disk_s3; } + ThrottlerPtr getPutRequestThrottler() const { return client_configuration.put_request_throttler; } + ThrottlerPtr getGetRequestThrottler() const { return client_configuration.get_request_throttler; } + private: friend struct ::MockS3::Client; diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 315202cc01d..20409577332 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -231,6 +231,31 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr if (unreserved) new_values[fmt::format("DiskUnreserved_{}", name)] = { *unreserved, "Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems may not provide this information." }; + + try + { + if (auto s3_client = disk->getS3StorageClient()) + { + if (auto put_throttler = s3_client->getPutRequestThrottler()) + { + new_values[fmt::format("DiskPutObjectThrottlerRPS_{}", name)] = { put_throttler->getMaxSpeed(), + "PutObject Request throttling limit on the disk in requests per second (virtual filesystem). Local filesystems may not provide this information." }; + new_values[fmt::format("DiskPutObjectThrottlerAvailable_{}", name)] = { put_throttler->getAvailable(), + "Number of PutObject requests that can be currently issued without hitting throttling limit on the disk (virtual filesystem). Local filesystems may not provide this information." }; + } + if (auto get_throttler = s3_client->getGetRequestThrottler()) + { + new_values[fmt::format("DiskGetObjectThrottlerRPS_{}", name)] = { get_throttler->getMaxSpeed(), + "GetObject Request throttling limit on the disk in requests per second (virtual filesystem). Local filesystems may not provide this information." }; + new_values[fmt::format("DiskGetObjectThrottlerAvailable_{}", name)] = { get_throttler->getAvailable(), + "Number of GetObject requests that can be currently issued without hitting throttling limit on the disk (virtual filesystem). Local filesystems may not provide this information." }; + } + } + } + catch(...) + { + // Skip disk than do not have s3 throttlers + } } } From 41cca9df1a251c36c2b4ff7f8469ceb7c9ca0251 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 10 Jun 2024 11:59:52 +0000 Subject: [PATCH 11/43] add space --- src/Interpreters/ServerAsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 20409577332..2892025da2d 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -252,7 +252,7 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr } } } - catch(...) + catch (...) { // Skip disk than do not have s3 throttlers } From 1fb5b35dd9a926de4eca1d83b7b1bfc29347d253 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 10 Jun 2024 13:53:25 +0000 Subject: [PATCH 12/43] explicitly define formatter for StringRef Signed-off-by: Duc Canh Le --- base/base/StringRef.h | 4 ++++ src/Coordination/KeeperSnapshotManager.cpp | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/base/base/StringRef.h b/base/base/StringRef.h index 24af84626de..fc0674b8440 100644 --- a/base/base/StringRef.h +++ b/base/base/StringRef.h @@ -12,6 +12,8 @@ #include #include #include +#include +#include #include @@ -376,3 +378,5 @@ namespace PackedZeroTraits std::ostream & operator<<(std::ostream & os, const StringRef & str); + +template<> struct fmt::formatter : fmt::ostream_formatter {}; diff --git a/src/Coordination/KeeperSnapshotManager.cpp b/src/Coordination/KeeperSnapshotManager.cpp index 23ff714a929..f25ccab86b1 100644 --- a/src/Coordination/KeeperSnapshotManager.cpp +++ b/src/Coordination/KeeperSnapshotManager.cpp @@ -466,7 +466,7 @@ void KeeperStorageSnapshot::deserialize(SnapshotDeserializationResult & deserial #ifdef NDEBUG /// TODO (alesapin) remove this, it should be always CORRUPTED_DATA. LOG_ERROR(getLogger("KeeperSnapshotManager"), "Children counter in stat.numChildren {}" - " is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key.toView()); + " is different from actual children size {} for node {}", itr.value.numChildren(), itr.value.getChildren().size(), itr.key); #else throw Exception(ErrorCodes::LOGICAL_ERROR, "Children counter in stat.numChildren {}" " is different from actual children size {} for node {}", From dcbcfc9ee84fe6eabdcd721b4d3fceace94ae371 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 10 Jun 2024 15:21:18 +0000 Subject: [PATCH 13/43] fix skipping of remote disks --- .../ServerAsynchronousMetrics.cpp | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 2892025da2d..5633b9b9350 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -210,28 +210,28 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr auto total = disk->getTotalSpace(); /// Some disks don't support information about the space. - if (!total) - continue; - - auto available = disk->getAvailableSpace(); - auto unreserved = disk->getUnreservedSpace(); - - new_values[fmt::format("DiskTotal_{}", name)] = { *total, - "The total size in bytes of the disk (virtual filesystem). Remote filesystems may not provide this information." }; - - if (available) + if (total) { - new_values[fmt::format("DiskUsed_{}", name)] = { *total - *available, - "Used bytes on the disk (virtual filesystem). Remote filesystems not always provide this information." }; + auto available = disk->getAvailableSpace(); + auto unreserved = disk->getUnreservedSpace(); - new_values[fmt::format("DiskAvailable_{}", name)] = { *available, - "Available bytes on the disk (virtual filesystem). Remote filesystems may not provide this information." }; + new_values[fmt::format("DiskTotal_{}", name)] = { *total, + "The total size in bytes of the disk (virtual filesystem). Remote filesystems may not provide this information." }; + + if (available) + { + new_values[fmt::format("DiskUsed_{}", name)] = { *total - *available, + "Used bytes on the disk (virtual filesystem). Remote filesystems not always provide this information." }; + + new_values[fmt::format("DiskAvailable_{}", name)] = { *available, + "Available bytes on the disk (virtual filesystem). Remote filesystems may not provide this information." }; + } + + if (unreserved) + new_values[fmt::format("DiskUnreserved_{}", name)] = { *unreserved, + "Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems may not provide this information." }; } - if (unreserved) - new_values[fmt::format("DiskUnreserved_{}", name)] = { *unreserved, - "Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems may not provide this information." }; - try { if (auto s3_client = disk->getS3StorageClient()) From 374854a972616bb42c0d2c960511e377fb271b8b Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 10 Jun 2024 18:20:55 +0200 Subject: [PATCH 14/43] Increase timeout in wait_for_all_mutations --- tests/queries/0_stateless/mergetree_mutations.lib | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/mergetree_mutations.lib b/tests/queries/0_stateless/mergetree_mutations.lib index b11b2e6b852..9eeea87b52d 100644 --- a/tests/queries/0_stateless/mergetree_mutations.lib +++ b/tests/queries/0_stateless/mergetree_mutations.lib @@ -37,7 +37,7 @@ function wait_for_all_mutations() echo "Timed out while waiting for mutation to execute!" fi - sleep 0.1 + sleep 0.3 done } From 32431b9c7d6569914724595cd69ba4178d26ccca Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Mon, 10 Jun 2024 18:40:57 +0200 Subject: [PATCH 15/43] Update src/Common/Throttler.cpp Co-authored-by: Antonio Andelic --- src/Common/Throttler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index 61d120e11ea..e7e38cdd6ad 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -104,8 +104,8 @@ bool Throttler::isThrottling() const Int64 Throttler::getAvailable() { // To update bucket state and receive current number of token in a thread-safe way - size_t count_value; - double tokens_value; + size_t count_value = 0; + double tokens_value = 0.0; addImpl(0, count_value, tokens_value); return static_cast(tokens_value); From eb839de79576f3bf554658009b33122d0fc27e98 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Mon, 10 Jun 2024 18:41:08 +0200 Subject: [PATCH 16/43] Update src/Interpreters/ServerAsynchronousMetrics.cpp Co-authored-by: Antonio Andelic --- src/Interpreters/ServerAsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 5633b9b9350..7defe2fc721 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -254,7 +254,7 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr } catch (...) { - // Skip disk than do not have s3 throttlers + // Skip disk that do not have s3 throttlers } } } From 0c8a9f099c9897b84b72197eb34b1bbc72a7bd78 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 10 Jun 2024 16:45:48 +0000 Subject: [PATCH 17/43] fix build --- src/Interpreters/ServerAsynchronousMetrics.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 5633b9b9350..9a0041eff05 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -232,6 +232,7 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr "Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems may not provide this information." }; } +#if USE_AWS_S3 try { if (auto s3_client = disk->getS3StorageClient()) @@ -256,6 +257,7 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr { // Skip disk than do not have s3 throttlers } +#endif } } From 65ee0254fa26a4bda05015d62a8250fa4d6ac1f1 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 10 Jun 2024 16:48:24 +0000 Subject: [PATCH 18/43] safer --- src/Common/Throttler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index e7e38cdd6ad..a581ff1766f 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -41,8 +41,8 @@ Throttler::Throttler(size_t max_speed_, size_t limit_, const char * limit_exceed UInt64 Throttler::add(size_t amount) { // Values obtained under lock to be checked after release - size_t count_value; - double tokens_value; + size_t count_value = 0; + double tokens_value = 0.0; addImpl(amount, count_value, tokens_value); if (limit && count_value > limit) From 1519efe7e21384b55780e214a5434dafdd1d1f63 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 11 Jun 2024 11:53:33 +0200 Subject: [PATCH 19/43] Addressed review comments --- src/Parsers/FunctionParameterValuesVisitor.cpp | 2 +- .../queries/0_stateless/03146_parameterized_view_with_date.sql | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Parsers/FunctionParameterValuesVisitor.cpp b/src/Parsers/FunctionParameterValuesVisitor.cpp index e791e07cdfb..eaf28bbbc41 100644 --- a/src/Parsers/FunctionParameterValuesVisitor.cpp +++ b/src/Parsers/FunctionParameterValuesVisitor.cpp @@ -23,7 +23,7 @@ class FunctionParameterValuesVisitor public: explicit FunctionParameterValuesVisitor(NameToNameMap & parameter_values_, ContextPtr context_) : parameter_values(parameter_values_) - ,context(context_) + , context(context_) { } diff --git a/tests/queries/0_stateless/03146_parameterized_view_with_date.sql b/tests/queries/0_stateless/03146_parameterized_view_with_date.sql index 53022e969ab..2cfadb70b24 100644 --- a/tests/queries/0_stateless/03146_parameterized_view_with_date.sql +++ b/tests/queries/0_stateless/03146_parameterized_view_with_date.sql @@ -9,4 +9,6 @@ create view pv as select * from table_pv where timestamp_field > {timestamp_para select * from pv (timestamp_param=toDateTime('2024-04-01 00:00:01')); +select * from pv (timestamp_param=toDateTime('2024-040')); -- { serverError CANNOT_PARSE_DATETIME } + drop table table_pv; From e0668ab8fa1fd90fc88c6d1088230279b6fb28b7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 11 Jun 2024 17:03:54 +0200 Subject: [PATCH 20/43] write more tests --- src/Interpreters/InterpreterCreateQuery.cpp | 3 +-- src/Storages/ColumnsDescription.cpp | 6 +++++- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 6 ++++++ src/Storages/StatisticsDescription.cpp | 4 ++-- src/Storages/StatisticsDescription.h | 2 +- .../test_manipulate_statistics/test.py | 18 ++++++++++++++++++ 6 files changed, 33 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a51d3e6dade..b2d65662eaa 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -680,8 +680,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( { if (!skip_checks && !context_->getSettingsRef().allow_experimental_statistics) throw Exception(ErrorCodes::INCORRECT_QUERY, "Create table with statistics is now disabled. Turn on allow_experimental_statistics"); - column.statistics = ColumnStatisticsDescription::fromColumnDeclaration(col_decl); - column.statistics.data_type = column.type; + column.statistics = ColumnStatisticsDescription::fromColumnDeclaration(col_decl, column.type); } if (col_decl.ttl) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 556f8a6e42d..c07583cd39d 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -209,7 +209,11 @@ void ColumnDescription::readText(ReadBuffer & buf) settings = col_ast->settings->as().changes; if (col_ast->statistics_desc) - statistics = ColumnStatisticsDescription::fromColumnDeclaration(*col_ast); + { + statistics = ColumnStatisticsDescription::fromColumnDeclaration(*col_ast, type); + /// every column has name `x` here, so we have to set the name manually. + statistics.column_name = name; + } } else throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description"); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 8e304936747..0dbddf05697 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -471,7 +471,13 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( ColumnsStatistics statistics; if (context->getSettingsRef().materialize_statistics_on_insert) + { + for (auto col : metadata_snapshot->getColumns()) + LOG_INFO(log, "column col {} stats {}", col.name, col.statistics.column_name); statistics = MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns()); + for (auto stats : statistics) + LOG_INFO(log, "writing stats {}", stats->columnName()); + } /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index fc06c66b50e..f10fb78f933 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -169,7 +169,7 @@ std::vector ColumnStatisticsDescription::fromAST(co return result; } -ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(const ASTColumnDeclaration & column) +ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type) { const auto & stat_type_list_ast = column.statistics_desc->as().arguments; if (stat_type_list_ast->children.empty()) @@ -185,7 +185,7 @@ ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(c throw Exception(ErrorCodes::INCORRECT_QUERY, "Column {} already contains statistics type {}", stats.column_name, stat_type); stats.types_to_desc.emplace(stat.type, std::move(stat)); } - + stats.data_type = data_type; return stats; } diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index 59ad8944850..4862fb79d45 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -55,7 +55,7 @@ struct ColumnStatisticsDescription ASTPtr getAST() const; static std::vector fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns); - static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column); + static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type); using StatisticsTypeDescMap = std::map; StatisticsTypeDescMap types_to_desc; diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index 53ab9682ad2..0ce90731e8d 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -155,3 +155,21 @@ def test_replicated_table_ddl(started_cluster): node2.query("SHOW CREATE TABLE test_stat") == "CREATE TABLE default.test_stat\\n(\\n `a` Int64 STATISTICS(tdigest, uniq),\\n `b` Int64,\\n `c` Int64 STATISTICS(tdigest, uniq)\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/test/statistics\\', \\'2\\')\\nORDER BY a\\nSETTINGS index_granularity = 8192\n" ) + + node2.query("insert into test_stat values(1,2,3), (2,3,4)"); + # check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "c", True) + node1.query("ALTER TABLE test_stat RENAME COLUMN c TO d", settings={"alter_sync": "2"}) + assert (node2.query("select sum(a), sum(d) from test_stat") == "3\t7\n") + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "a", True) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "c", False) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "d", True) + node1.query("ALTER TABLE test_stat CLEAR STATISTICS d", settings={"alter_sync": "2"}) + node1.query("ALTER TABLE test_stat ADD STATISTICS b type tdigest", settings={"alter_sync": "2"}) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "a", True) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "b", False) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "d", False) + node1.query("ALTER TABLE test_stat MATERIALIZE STATISTICS b", settings={"alter_sync": "2"}) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "a", True) + check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "b", True) From d8366119c403429c2748f4ea372d0542701147f6 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 11 Jun 2024 22:04:37 +0100 Subject: [PATCH 21/43] impl --- src/Processors/QueryPlan/ReadFromLoopStep.cpp | 26 ++++++++++++------- ...op_engine_with_parallel_replicas.reference | 0 ...168_loop_engine_with_parallel_replicas.sql | 9 +++++++ 3 files changed, 26 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.reference create mode 100644 tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql diff --git a/src/Processors/QueryPlan/ReadFromLoopStep.cpp b/src/Processors/QueryPlan/ReadFromLoopStep.cpp index 10436490a2a..2e5fa3ec9f7 100644 --- a/src/Processors/QueryPlan/ReadFromLoopStep.cpp +++ b/src/Processors/QueryPlan/ReadFromLoopStep.cpp @@ -1,14 +1,15 @@ -#include -#include -#include -#include -#include +#include +#include +#include #include #include -#include -#include +#include +#include #include -#include +#include +#include +#include +#include namespace DB { @@ -111,6 +112,13 @@ namespace DB std::unique_ptr executor; }; + static ContextPtr disableParallelReplicas(ContextPtr context) + { + auto modified_context = Context::createCopy(context); + modified_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); + return modified_context; + } + ReadFromLoopStep::ReadFromLoopStep( const Names & column_names_, const SelectQueryInfo & query_info_, @@ -125,7 +133,7 @@ namespace DB column_names_, query_info_, storage_snapshot_, - context_) + disableParallelReplicas(context_)) , column_names(column_names_) , processed_stage(processed_stage_) , inner_storage(std::move(inner_storage_)) diff --git a/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.reference b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql new file mode 100644 index 00000000000..dfcb5de9f2a --- /dev/null +++ b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql @@ -0,0 +1,9 @@ +DROP DATABASE IF EXISTS 03147_db; +CREATE DATABASE IF NOT EXISTS 03147_db; +CREATE TABLE 03147_db.t (n Int8) ENGINE=MergeTree ORDER BY n; +INSERT INTO 03147_db.t SELECT * FROM numbers(10); +USE 03147_db; + +SET allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 100; + +SELECT * FROM loop(03147_db.t) LIMIT 15 FORMAT Null; From f632424f92dbb242369d5a17a6788ea57d9d0103 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 12 Jun 2024 04:32:34 +0000 Subject: [PATCH 22/43] remove unnecessary change Signed-off-by: Duc Canh Le --- src/Common/filesystemHelpers.cpp | 4 +--- src/Functions/FunctionsStringDistance.cpp | 4 ++-- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Common/filesystemHelpers.cpp b/src/Common/filesystemHelpers.cpp index 09c4508b7b2..2d053c615d9 100644 --- a/src/Common/filesystemHelpers.cpp +++ b/src/Common/filesystemHelpers.cpp @@ -1,6 +1,4 @@ #include "filesystemHelpers.h" -#include -#include #if defined(OS_LINUX) # include @@ -13,7 +11,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Functions/FunctionsStringDistance.cpp b/src/Functions/FunctionsStringDistance.cpp index d0d8ebc946d..6cb23bbea9f 100644 --- a/src/Functions/FunctionsStringDistance.cpp +++ b/src/Functions/FunctionsStringDistance.cpp @@ -159,7 +159,7 @@ struct ByteJaccardIndexImpl } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", std::string_view(haystack, haystack_end - haystack)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", StringRef(haystack, haystack_end - haystack)); } } } @@ -186,7 +186,7 @@ struct ByteJaccardIndexImpl } else { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", std::string_view(needle, needle_end - needle)); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Illegal UTF-8 sequence, while processing '{}'", StringRef(needle, needle_end - needle)); } } } From af4541755a43fcacf1a1a58edcf6b07bd5fcedec Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 12 Jun 2024 14:27:13 +0200 Subject: [PATCH 23/43] fix black --- .../test_manipulate_statistics/test.py | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index 0ce90731e8d..9485b611c01 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -156,20 +156,29 @@ def test_replicated_table_ddl(started_cluster): == "CREATE TABLE default.test_stat\\n(\\n `a` Int64 STATISTICS(tdigest, uniq),\\n `b` Int64,\\n `c` Int64 STATISTICS(tdigest, uniq)\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/test/statistics\\', \\'2\\')\\nORDER BY a\\nSETTINGS index_granularity = 8192\n" ) - node2.query("insert into test_stat values(1,2,3), (2,3,4)"); + node2.query("insert into test_stat values(1,2,3), (2,3,4)") # check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "c", True) - node1.query("ALTER TABLE test_stat RENAME COLUMN c TO d", settings={"alter_sync": "2"}) - assert (node2.query("select sum(a), sum(d) from test_stat") == "3\t7\n") + node1.query( + "ALTER TABLE test_stat RENAME COLUMN c TO d", settings={"alter_sync": "2"} + ) + assert node2.query("select sum(a), sum(d) from test_stat") == "3\t7\n" check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "c", False) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "d", True) - node1.query("ALTER TABLE test_stat CLEAR STATISTICS d", settings={"alter_sync": "2"}) - node1.query("ALTER TABLE test_stat ADD STATISTICS b type tdigest", settings={"alter_sync": "2"}) + node1.query( + "ALTER TABLE test_stat CLEAR STATISTICS d", settings={"alter_sync": "2"} + ) + node1.query( + "ALTER TABLE test_stat ADD STATISTICS b type tdigest", + settings={"alter_sync": "2"}, + ) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "b", False) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "d", False) - node1.query("ALTER TABLE test_stat MATERIALIZE STATISTICS b", settings={"alter_sync": "2"}) + node1.query( + "ALTER TABLE test_stat MATERIALIZE STATISTICS b", settings={"alter_sync": "2"} + ) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "b", True) From d798fffb1d772266095f599173212874a3897ab1 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 12 Jun 2024 16:46:14 +0200 Subject: [PATCH 24/43] fix --- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 0dbddf05697..8e304936747 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -471,13 +471,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( ColumnsStatistics statistics; if (context->getSettingsRef().materialize_statistics_on_insert) - { - for (auto col : metadata_snapshot->getColumns()) - LOG_INFO(log, "column col {} stats {}", col.name, col.statistics.column_name); statistics = MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns()); - for (auto stats : statistics) - LOG_INFO(log, "writing stats {}", stats->columnName()); - } /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) From 35252c4eda20d6dfcb878d7b1522e7842d826f60 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 12 Jun 2024 18:41:46 +0100 Subject: [PATCH 25/43] impl --- .../0_stateless/03168_loop_engine_with_parallel_replicas.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql index dfcb5de9f2a..da4626ad897 100644 --- a/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql +++ b/tests/queries/0_stateless/03168_loop_engine_with_parallel_replicas.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel + DROP DATABASE IF EXISTS 03147_db; CREATE DATABASE IF NOT EXISTS 03147_db; CREATE TABLE 03147_db.t (n Int8) ENGINE=MergeTree ORDER BY n; From 6d48962ca09f582478e6caa160463b34ead0091b Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Mon, 10 Jun 2024 23:54:19 +0100 Subject: [PATCH 26/43] impl --- .../QueryPlan/ReadFromMergeTree.cpp | 9 ++++ src/Storages/StorageMerge.cpp | 6 +++ .../03155_test_move_to_prewhere.reference | 1 + .../03155_test_move_to_prewhere.sh | 46 +++++++++++++++++++ 4 files changed, 62 insertions(+) create mode 100644 tests/queries/0_stateless/03155_test_move_to_prewhere.reference create mode 100755 tests/queries/0_stateless/03155_test_move_to_prewhere.sh diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index e469062d7e7..4fad1bbb653 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1515,6 +1515,15 @@ static void buildIndexes( void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) { + /// Sometimes a really dumb problem may happen. + /// For ReadFromMerge for example we may recursively call `applyFilters` for child reading steps (with no filters added so far). + /// Then later `optimizePrimaryKeyCondition` will try to apply filters to those child reading steps, but with no luck, + /// because we already made an `applyFilters` call that could lead to indexes initialization few lines below. + /// So effectively the right set of filters will be just ignored. + /// This is not an ultimate solution, of course, we're better to have more structured way of applying filters. + if (added_filter_nodes.nodes.empty()) + return; + if (!indexes) { filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes, query_info.buildNodeNameToInputNodeColumn()); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index ed3f43367dd..55cfd1ffcd7 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1573,8 +1573,14 @@ QueryPlanRawPtrs ReadFromMerge::getChildPlans() QueryPlanRawPtrs plans; for (auto & child_plan : *child_plans) + { if (child_plan.plan.isInitialized()) + { + /// So we will see the optimized plan in EXPLAIN output + child_plan.plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); plans.push_back(&child_plan.plan); + } + } return plans; } diff --git a/tests/queries/0_stateless/03155_test_move_to_prewhere.reference b/tests/queries/0_stateless/03155_test_move_to_prewhere.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/03155_test_move_to_prewhere.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/03155_test_move_to_prewhere.sh b/tests/queries/0_stateless/03155_test_move_to_prewhere.sh new file mode 100755 index 00000000000..b6980b3a23a --- /dev/null +++ b/tests/queries/0_stateless/03155_test_move_to_prewhere.sh @@ -0,0 +1,46 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -nq " + CREATE TABLE event_envoy + ( + timestamp_interval DateTime CODEC(DoubleDelta), + region LowCardinality(String), + cluster LowCardinality(String) + ) + ENGINE = MergeTree + ORDER BY (timestamp_interval) + SETTINGS index_granularity = 8192; + + INSERT INTO event_envoy SELECT now() - number, 'us-east-1', 'ch_super_fast' FROM numbers_mt(1e5); +" + +${CLICKHOUSE_CLIENT} -nq " + CREATE TABLE event_envoy_remote + ( + timestamp_interval DateTime CODEC(DoubleDelta), + region LowCardinality(String), + cluster LowCardinality(String) + ) AS remote('127.0.0.1', '${CLICKHOUSE_DATABASE}', event_envoy); +" + +${CLICKHOUSE_CLIENT} -q " + CREATE TABLE global_event_envoy + ( + timestamp_interval DateTime, + region LowCardinality(String), + cluster LowCardinality(String) + ) + ENGINE = Merge('${CLICKHOUSE_DATABASE}', 'event_envoy.*'); +" + +${CLICKHOUSE_CLIENT} --prefer_localhost_replica 1 -q " + EXPLAIN indexes=1 + SELECT timestamp_interval + FROM global_event_envoy + WHERE timestamp_interval <= now() - 54321 AND region = 'us-east-1' +" | grep -c 'Condition.*timestamp_interval' + From 55e0c668ae58ada0071916853aa07952c51fa15d Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Wed, 12 Jun 2024 20:49:26 +0100 Subject: [PATCH 27/43] fix --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 9 --------- src/Storages/StorageMerge.cpp | 6 ------ 2 files changed, 15 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 4fad1bbb653..e469062d7e7 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1515,15 +1515,6 @@ static void buildIndexes( void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes) { - /// Sometimes a really dumb problem may happen. - /// For ReadFromMerge for example we may recursively call `applyFilters` for child reading steps (with no filters added so far). - /// Then later `optimizePrimaryKeyCondition` will try to apply filters to those child reading steps, but with no luck, - /// because we already made an `applyFilters` call that could lead to indexes initialization few lines below. - /// So effectively the right set of filters will be just ignored. - /// This is not an ultimate solution, of course, we're better to have more structured way of applying filters. - if (added_filter_nodes.nodes.empty()) - return; - if (!indexes) { filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes, query_info.buildNodeNameToInputNodeColumn()); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 55cfd1ffcd7..ed3f43367dd 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -1573,14 +1573,8 @@ QueryPlanRawPtrs ReadFromMerge::getChildPlans() QueryPlanRawPtrs plans; for (auto & child_plan : *child_plans) - { if (child_plan.plan.isInitialized()) - { - /// So we will see the optimized plan in EXPLAIN output - child_plan.plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); plans.push_back(&child_plan.plan); - } - } return plans; } From 857a412e3b274bb5a309bebe9bfd284ae5ac8ad8 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Thu, 13 Jun 2024 01:27:54 +0000 Subject: [PATCH 28/43] address some review comments Signed-off-by: Duc Canh Le --- src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp | 4 ++-- src/Storages/Kafka/KafkaConsumer.h | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp index 56c774782c2..6a3475a1830 100644 --- a/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BSONEachRowRowInputFormat.cpp @@ -421,7 +421,7 @@ void BSONEachRowRowInputFormat::readTuple(IColumn & column, const DataTypePtr & "Cannot parse tuple column with type {} from BSON array/embedded document field: " "tuple doesn't have element with name \"{}\"", data_type->getName(), - name.toView()); + name); index = *try_get_index; } @@ -806,7 +806,7 @@ bool BSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi else { if (seen_columns[index]) - throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing BSONEachRow format: {}", name.toView()); + throw Exception(ErrorCodes::INCORRECT_DATA, "Duplicate field found while parsing BSONEachRow format: {}", name); seen_columns[index] = true; read_columns[index] = readField(*columns[index], types[index], BSONType(type)); diff --git a/src/Storages/Kafka/KafkaConsumer.h b/src/Storages/Kafka/KafkaConsumer.h index a2d047933be..4daf8652c3b 100644 --- a/src/Storages/Kafka/KafkaConsumer.h +++ b/src/Storages/Kafka/KafkaConsumer.h @@ -1,14 +1,14 @@ #pragma once -#include -#include #include +#include #include #include #include #include +#include #include namespace CurrentMetrics From 7c7be37383e015da0302708a9cd43d2ff7361496 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 13 Jun 2024 13:49:03 +0000 Subject: [PATCH 29/43] fix tidy build --- src/Interpreters/ServerAsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 6afd3b6df3a..83ff025d2a6 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -253,7 +253,7 @@ void ServerAsynchronousMetrics::updateImpl(TimePoint update_time, TimePoint curr } } } - catch (...) + catch (...) // NOLINT(bugprone-empty-catch) { // Skip disk that do not have s3 throttlers } From a6f05df749e8c6720172db2b4e09899a9b5a0498 Mon Sep 17 00:00:00 2001 From: unashi Date: Wed, 5 Jun 2024 19:46:23 +0800 Subject: [PATCH 30/43] [feature] Add an asynchronous_metric jemalloc.profile.active to show whether sampling is currently active for the calling thread. This is an activation mechanism in addition to prof.active; both must be active for the calling thread to sample. --- .../system-tables/asynchronous_metrics.md | 4 ++ src/Common/AsynchronousMetrics.cpp | 10 ++++ .../__init__.py | 0 .../asynchronous_metrics_update_period_s.xml | 3 ++ .../test.py | 49 +++++++++++++++++++ 5 files changed, 66 insertions(+) create mode 100644 tests/integration/test_asynchronous_metric_jemalloc_profile_active/__init__.py create mode 100644 tests/integration/test_asynchronous_metric_jemalloc_profile_active/configs/asynchronous_metrics_update_period_s.xml create mode 100644 tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py diff --git a/docs/en/operations/system-tables/asynchronous_metrics.md b/docs/en/operations/system-tables/asynchronous_metrics.md index 81725b97e41..762d187917c 100644 --- a/docs/en/operations/system-tables/asynchronous_metrics.md +++ b/docs/en/operations/system-tables/asynchronous_metrics.md @@ -639,6 +639,10 @@ An internal metric of the low-level memory allocator (jemalloc). See https://jem An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html +### jemalloc.prof.active + +An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html + **See Also** - [Monitoring](../../operations/monitoring.md) — Base concepts of ClickHouse monitoring. diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 4c71b9846c7..d6c8b36f171 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -415,6 +415,15 @@ Value saveAllArenasMetric(AsynchronousMetricValues & values, fmt::format("jemalloc.arenas.all.{}", metric_name)); } +template +static Value saveJemallocProf(AsynchronousMetricValues & values, + const std::string & metric_name) +{ + return saveJemallocMetricImpl(values, + fmt::format("prof.{}", metric_name), + fmt::format("jemalloc.prof.{}", metric_name)); +} + } #endif @@ -607,6 +616,7 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) saveJemallocMetric(new_values, "background_thread.num_threads"); saveJemallocMetric(new_values, "background_thread.num_runs"); saveJemallocMetric(new_values, "background_thread.run_intervals"); + saveJemallocProf(new_values, "active"); saveAllArenasMetric(new_values, "pactive"); [[maybe_unused]] size_t je_malloc_pdirty = saveAllArenasMetric(new_values, "pdirty"); [[maybe_unused]] size_t je_malloc_pmuzzy = saveAllArenasMetric(new_values, "pmuzzy"); diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/__init__.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/configs/asynchronous_metrics_update_period_s.xml b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/configs/asynchronous_metrics_update_period_s.xml new file mode 100644 index 00000000000..47e88730482 --- /dev/null +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/configs/asynchronous_metrics_update_period_s.xml @@ -0,0 +1,3 @@ + + 1 + diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py new file mode 100644 index 00000000000..245b1fd3bb9 --- /dev/null +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -0,0 +1,49 @@ +import time + +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance( + "node1", + main_configs=["configs/asynchronous_metrics_update_period_s.xml"], + env_variables={"MALLOC_CONF": "background_thread:true,prof:true"}, +) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +# Tests that the system.asynchronous_metric_log table gets populated. +# asynchronous metrics are updated once every 60s by default. To make the test run faster, the setting +# asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and +# are available for querying during the test. +def test_event_time_microseconds_field(started_cluster): + res_t = node1.query("SYSTEM JEMALLOC ENABLE PROFILE") + res_o = node1.query("SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;") + assert ( + res_o== """Row 1: +────── +metric: jemalloc.prof.active +value: 1 +description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html +""" + ) + node1.query("SYSTEM JEMALLOC DISABLE PROFILE") + time.sleep(5) + res_t = node1.query("SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;") + assert ( + res_t== """Row 1: +────── +metric: jemalloc.prof.active +value: 0 +description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html +""" + ) From 83359af8b75c4d23ff89e71b1f9130027588b735 Mon Sep 17 00:00:00 2001 From: unashi Date: Wed, 5 Jun 2024 20:38:18 +0800 Subject: [PATCH 31/43] [update] rm useless res --- .../test_asynchronous_metric_jemalloc_profile_active/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py index 245b1fd3bb9..fe0ff46cedb 100644 --- a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -26,7 +26,7 @@ def started_cluster(): # asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and # are available for querying during the test. def test_event_time_microseconds_field(started_cluster): - res_t = node1.query("SYSTEM JEMALLOC ENABLE PROFILE") + node1.query("SYSTEM JEMALLOC ENABLE PROFILE") res_o = node1.query("SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;") assert ( res_o== """Row 1: From 761d8e327c3c27b246aefce15afb7061aeb986e7 Mon Sep 17 00:00:00 2001 From: unashi Date: Thu, 6 Jun 2024 10:20:09 +0800 Subject: [PATCH 32/43] [update] 1. black test.py 2. adjust test.py to cover all situation --- .../test.py | 32 ++++++++++++++++--- 1 file changed, 27 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py index fe0ff46cedb..80165a28c76 100644 --- a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -26,24 +26,46 @@ def started_cluster(): # asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and # are available for querying during the test. def test_event_time_microseconds_field(started_cluster): - node1.query("SYSTEM JEMALLOC ENABLE PROFILE") - res_o = node1.query("SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;") + # prof:true -> default open + res_o = node1.query( + "SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;" + ) assert ( - res_o== """Row 1: + res_o + == """Row 1: ────── metric: jemalloc.prof.active value: 1 description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html """ ) + # disable node1.query("SYSTEM JEMALLOC DISABLE PROFILE") time.sleep(5) - res_t = node1.query("SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;") + res_t = node1.query( + "SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;" + ) assert ( - res_t== """Row 1: + res_t + == """Row 1: ────── metric: jemalloc.prof.active value: 0 description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html +""" + ) + # enable + node1.query("SYSTEM JEMALLOC ENABLE PROFILE") + time.sleep(5) + res_f = node1.query( + "SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;" + ) + assert ( + res_f + == """Row 1: +────── +metric: jemalloc.prof.active +value: 1 +description: An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html """ ) From 13bdcc335f03c5e6a40c285029acc646432227e2 Mon Sep 17 00:00:00 2001 From: unashi Date: Thu, 6 Jun 2024 17:44:20 +0800 Subject: [PATCH 33/43] [fix] remove static from function --- src/Common/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index d6c8b36f171..6309f6079f6 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -416,7 +416,7 @@ Value saveAllArenasMetric(AsynchronousMetricValues & values, } template -static Value saveJemallocProf(AsynchronousMetricValues & values, +Value saveJemallocProf(AsynchronousMetricValues & values, const std::string & metric_name) { return saveJemallocMetricImpl(values, From 17b03c7df9d23e0a6656a9652e73e6ea5fecf251 Mon Sep 17 00:00:00 2001 From: unashi Date: Sat, 8 Jun 2024 00:41:47 +0800 Subject: [PATCH 34/43] [fix] skip sanitizers --- .../test_asynchronous_metric_jemalloc_profile_active/test.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py index 80165a28c76..1283becca6e 100644 --- a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -27,6 +27,9 @@ def started_cluster(): # are available for querying during the test. def test_event_time_microseconds_field(started_cluster): # prof:true -> default open + if node1.is_built_with_sanitizer(): + pytest.skip("Disabled for sanitizers") + res_o = node1.query( "SELECT * FROM system.asynchronous_metrics WHERE metric ILIKE '%jemalloc.prof.active%' FORMAT Vertical;" ) From 3c8f3c1930d126295544d46dd88ad912dad74174 Mon Sep 17 00:00:00 2001 From: unashi Date: Sat, 8 Jun 2024 00:53:44 +0800 Subject: [PATCH 35/43] [fix] fix the name of function in test.py --- .../test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py index 1283becca6e..218b3e2ec6a 100644 --- a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -21,11 +21,11 @@ def started_cluster(): cluster.shutdown() -# Tests that the system.asynchronous_metric_log table gets populated. + # asynchronous metrics are updated once every 60s by default. To make the test run faster, the setting -# asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and +# asynchronous_metric_update_period_s is being set to 1s so that the metrics are populated faster and # are available for querying during the test. -def test_event_time_microseconds_field(started_cluster): +def test_asynchronous_metric_jemalloc_profile_active(started_cluster): # prof:true -> default open if node1.is_built_with_sanitizer(): pytest.skip("Disabled for sanitizers") From f707c0d1ebfc1cd67ba5c5b76f2c055f4ab23a07 Mon Sep 17 00:00:00 2001 From: unashi Date: Sat, 8 Jun 2024 13:52:13 +0800 Subject: [PATCH 36/43] [black] --- .../test_asynchronous_metric_jemalloc_profile_active/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py index 218b3e2ec6a..627285a2038 100644 --- a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -21,7 +21,6 @@ def started_cluster(): cluster.shutdown() - # asynchronous metrics are updated once every 60s by default. To make the test run faster, the setting # asynchronous_metric_update_period_s is being set to 1s so that the metrics are populated faster and # are available for querying during the test. From 06e11752fde54b4d14ac0634b32646496e5039db Mon Sep 17 00:00:00 2001 From: unashi Date: Sat, 8 Jun 2024 18:19:15 +0800 Subject: [PATCH 37/43] [retry test] --- .../test_asynchronous_metric_jemalloc_profile_active/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py index 627285a2038..a8f4ab05888 100644 --- a/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py +++ b/tests/integration/test_asynchronous_metric_jemalloc_profile_active/test.py @@ -25,7 +25,7 @@ def started_cluster(): # asynchronous_metric_update_period_s is being set to 1s so that the metrics are populated faster and # are available for querying during the test. def test_asynchronous_metric_jemalloc_profile_active(started_cluster): - # prof:true -> default open + # default open if node1.is_built_with_sanitizer(): pytest.skip("Disabled for sanitizers") From 67c41d0856d7ff653820d376359eb455376d0a5f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jun 2024 11:50:52 +0200 Subject: [PATCH 38/43] Add a test just in case --- .../queries/0_stateless/03171_condition_pushdown.reference | 2 ++ tests/queries/0_stateless/03171_condition_pushdown.sql | 6 ++++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/03171_condition_pushdown.reference create mode 100644 tests/queries/0_stateless/03171_condition_pushdown.sql diff --git a/tests/queries/0_stateless/03171_condition_pushdown.reference b/tests/queries/0_stateless/03171_condition_pushdown.reference new file mode 100644 index 00000000000..c7e6479af3b --- /dev/null +++ b/tests/queries/0_stateless/03171_condition_pushdown.reference @@ -0,0 +1,2 @@ +123 123 +456 456 diff --git a/tests/queries/0_stateless/03171_condition_pushdown.sql b/tests/queries/0_stateless/03171_condition_pushdown.sql new file mode 100644 index 00000000000..9cfe41ce921 --- /dev/null +++ b/tests/queries/0_stateless/03171_condition_pushdown.sql @@ -0,0 +1,6 @@ +-- This query succeeds only if it is correctly optimized. +SET allow_experimental_analyzer = 1; +SELECT * FROM (SELECT * FROM numbers(1e19)) AS t1, (SELECT * FROM numbers(1e19)) AS t2 WHERE t1.number IN (123, 456) AND t2.number = t1.number ORDER BY ALL; + +-- Still TODO: +-- SELECT * FROM (SELECT * FROM numbers(1e19)) AS t1, (SELECT * FROM numbers(1e19)) AS t2 WHERE t1.number IN (SELECT 123 UNION ALL SELECT 456) AND t2.number = t1.number ORDER BY ALL; From aacb85495b826d26c84a17b28c3a479124aaf319 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 14 Jun 2024 13:58:04 +0200 Subject: [PATCH 39/43] Update tests/integration/test_manipulate_statistics/test.py --- tests/integration/test_manipulate_statistics/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index 9485b611c01..a602cce63df 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -157,7 +157,6 @@ def test_replicated_table_ddl(started_cluster): ) node2.query("insert into test_stat values(1,2,3), (2,3,4)") - # check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "c", True) node1.query( From 36a575e124a3fe940cfcecb565270784b0f89ee4 Mon Sep 17 00:00:00 2001 From: Blargian Date: Fri, 14 Jun 2024 15:55:02 +0200 Subject: [PATCH 40/43] Fix float page --- docs/en/sql-reference/data-types/float.md | 32 +++++++++++++++-------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/data-types/float.md b/docs/en/sql-reference/data-types/float.md index 23131d5b4fe..7c232a286ea 100644 --- a/docs/en/sql-reference/data-types/float.md +++ b/docs/en/sql-reference/data-types/float.md @@ -7,33 +7,43 @@ sidebar_label: Float32, Float64 # Float32, Float64 :::note -If you need accurate calculations, in particular if you work with financial or business data requiring a high precision you should consider using Decimal instead. Floats might lead to inaccurate results as illustrated below: +If you need accurate calculations, in particular if you work with financial or business data requiring a high precision, you should consider using [Decimal](../data-types/decimal.md) instead. -``` +[Floating Point Numbers](https://en.wikipedia.org/wiki/IEEE_754) might lead to inaccurate results as illustrated below: + +```sql CREATE TABLE IF NOT EXISTS float_vs_decimal ( my_float Float64, my_decimal Decimal64(3) -)Engine=MergeTree ORDER BY tuple() - -INSERT INTO float_vs_decimal SELECT round(randCanonical(), 3) AS res, res FROM system.numbers LIMIT 1000000; # Generate 1 000 000 random number with 2 decimal places and store them as a float and as a decimal +) +Engine=MergeTree +ORDER BY tuple(); +# Generate 1 000 000 random number with 2 decimal places and store them as a float and as a decimal +INSERT INTO float_vs_decimal SELECT round(randCanonical(), 3) AS res, res FROM system.numbers LIMIT 1000000; +``` +``` SELECT sum(my_float), sum(my_decimal) FROM float_vs_decimal; -> 500279.56300000014 500279.563 + +┌──────sum(my_float)─┬─sum(my_decimal)─┐ +│ 499693.60500000004 │ 499693.605 │ +└────────────────────┴─────────────────┘ SELECT sumKahan(my_float), sumKahan(my_decimal) FROM float_vs_decimal; -> 500279.563 500279.563 + +┌─sumKahan(my_float)─┬─sumKahan(my_decimal)─┐ +│ 499693.605 │ 499693.605 │ +└────────────────────┴──────────────────────┘ ``` ::: -[Floating point numbers](https://en.wikipedia.org/wiki/IEEE_754). - -Types are equivalent to types of C: +The equivalent types in ClickHouse and in C are given below: - `Float32` — `float`. - `Float64` — `double`. -Aliases: +Float types in ClickHouse have the following aliases: - `Float32` — `FLOAT`, `REAL`, `SINGLE`. - `Float64` — `DOUBLE`, `DOUBLE PRECISION`. From ad103b9db8dfeec41862ccb18acfe908b5b0d14b Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Fri, 14 Jun 2024 16:06:40 +0200 Subject: [PATCH 41/43] Update float.md --- docs/en/sql-reference/data-types/float.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/float.md b/docs/en/sql-reference/data-types/float.md index 7c232a286ea..3c789076c1e 100644 --- a/docs/en/sql-reference/data-types/float.md +++ b/docs/en/sql-reference/data-types/float.md @@ -20,7 +20,7 @@ CREATE TABLE IF NOT EXISTS float_vs_decimal Engine=MergeTree ORDER BY tuple(); -# Generate 1 000 000 random number with 2 decimal places and store them as a float and as a decimal +# Generate 1 000 000 random numbers with 2 decimal places and store them as a float and as a decimal INSERT INTO float_vs_decimal SELECT round(randCanonical(), 3) AS res, res FROM system.numbers LIMIT 1000000; ``` ``` From d421d39b735b29b348d21a607b8de61ecfaf3994 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 14 Jun 2024 15:33:31 +0000 Subject: [PATCH 42/43] Update version_date.tsv and changelogs after v23.8.15.35-lts --- docs/changelogs/v23.8.15.35-lts.md | 40 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 41 insertions(+) create mode 100644 docs/changelogs/v23.8.15.35-lts.md diff --git a/docs/changelogs/v23.8.15.35-lts.md b/docs/changelogs/v23.8.15.35-lts.md new file mode 100644 index 00000000000..bab5c507fe8 --- /dev/null +++ b/docs/changelogs/v23.8.15.35-lts.md @@ -0,0 +1,40 @@ +--- +sidebar_position: 1 +sidebar_label: 2024 +--- + +# 2024 Changelog + +### ClickHouse release v23.8.15.35-lts (060ff8e813a) FIXME as compared to v23.8.14.6-lts (967e51c1d6b) + +#### Build/Testing/Packaging Improvement +* Backported in [#63621](https://github.com/ClickHouse/ClickHouse/issues/63621): The Dockerfile is reviewed by the docker official library in https://github.com/docker-library/official-images/pull/15846. [#63400](https://github.com/ClickHouse/ClickHouse/pull/63400) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#65153](https://github.com/ClickHouse/ClickHouse/issues/65153): Decrease the `unit-test` image a few times. [#65102](https://github.com/ClickHouse/ClickHouse/pull/65102) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### Bug Fix (user-visible misbehavior in an official stable release) + +* Backported in [#64422](https://github.com/ClickHouse/ClickHouse/issues/64422): Fixes [#59989](https://github.com/ClickHouse/ClickHouse/issues/59989): runs init scripts when force-enabled or when no database exists, rather than the inverse. [#59991](https://github.com/ClickHouse/ClickHouse/pull/59991) ([jktng](https://github.com/jktng)). +* Backported in [#64016](https://github.com/ClickHouse/ClickHouse/issues/64016): Fix "Invalid storage definition in metadata file" for parameterized views. [#60708](https://github.com/ClickHouse/ClickHouse/pull/60708) ([Azat Khuzhin](https://github.com/azat)). +* Backported in [#63456](https://github.com/ClickHouse/ClickHouse/issues/63456): Fix the issue where the function `addDays` (and similar functions) reports an error when the first parameter is `DateTime64`. [#61561](https://github.com/ClickHouse/ClickHouse/pull/61561) ([Shuai li](https://github.com/loneylee)). +* Backported in [#63289](https://github.com/ClickHouse/ClickHouse/issues/63289): Fix crash with untuple and unresolved lambda. [#63131](https://github.com/ClickHouse/ClickHouse/pull/63131) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#63512](https://github.com/ClickHouse/ClickHouse/issues/63512): Fix `X-ClickHouse-Timezone` header returning wrong timezone when using `session_timezone` as query level setting. [#63377](https://github.com/ClickHouse/ClickHouse/pull/63377) ([Andrey Zvonov](https://github.com/zvonand)). +* Backported in [#63902](https://github.com/ClickHouse/ClickHouse/issues/63902): `query_plan_remove_redundant_distinct` can break queries with WINDOW FUNCTIONS (with `allow_experimental_analyzer` is on). Fixes [#62820](https://github.com/ClickHouse/ClickHouse/issues/62820). [#63776](https://github.com/ClickHouse/ClickHouse/pull/63776) ([Igor Nikonov](https://github.com/devcrafter)). +* Backported in [#64104](https://github.com/ClickHouse/ClickHouse/issues/64104): Deserialize untrusted binary inputs in a safer way. [#64024](https://github.com/ClickHouse/ClickHouse/pull/64024) ([Robert Schulze](https://github.com/rschu1ze)). +* Backported in [#64265](https://github.com/ClickHouse/ClickHouse/issues/64265): Prevent LOGICAL_ERROR on CREATE TABLE as MaterializedView. [#64174](https://github.com/ClickHouse/ClickHouse/pull/64174) ([Raúl Marín](https://github.com/Algunenano)). +* Backported in [#64867](https://github.com/ClickHouse/ClickHouse/issues/64867): Fixed memory possible incorrect memory tracking in several kinds of queries: queries that read any data from S3, queries via http protocol, asynchronous inserts. [#64844](https://github.com/ClickHouse/ClickHouse/pull/64844) ([Anton Popov](https://github.com/CurtizJ)). + +#### NO CL CATEGORY + +* Backported in [#63704](https://github.com/ClickHouse/ClickHouse/issues/63704):. [#63415](https://github.com/ClickHouse/ClickHouse/pull/63415) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + +#### NO CL ENTRY + +* NO CL ENTRY: 'Installation test has wrong check_state'. [#63994](https://github.com/ClickHouse/ClickHouse/pull/63994) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Backported in [#63343](https://github.com/ClickHouse/ClickHouse/issues/63343): The commit url has different pattern. [#63331](https://github.com/ClickHouse/ClickHouse/pull/63331) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Backported in [#63965](https://github.com/ClickHouse/ClickHouse/issues/63965): fix 02124_insert_deduplication_token_multiple_blocks. [#63950](https://github.com/ClickHouse/ClickHouse/pull/63950) ([Han Fei](https://github.com/hanfei1991)). +* Backported in [#64043](https://github.com/ClickHouse/ClickHouse/issues/64043): Do not create new release in release branch automatically. [#64039](https://github.com/ClickHouse/ClickHouse/pull/64039) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Pin requests to fix the integration tests. [#65183](https://github.com/ClickHouse/ClickHouse/pull/65183) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 719c25bdc95..1bff4cb0b09 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -41,6 +41,7 @@ v23.9.4.11-stable 2023-11-08 v23.9.3.12-stable 2023-10-31 v23.9.2.56-stable 2023-10-19 v23.9.1.1854-stable 2023-09-29 +v23.8.15.35-lts 2024-06-14 v23.8.14.6-lts 2024-05-02 v23.8.13.25-lts 2024-04-26 v23.8.12.13-lts 2024-03-26 From 799e5730082fb2060171b93ae25e735054d5308b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 14 Jun 2024 19:03:32 +0200 Subject: [PATCH 43/43] Fix silly typo that caused wrong tags messages --- tests/ci/release.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index 7490ce0a373..7ab1149310f 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -550,7 +550,7 @@ class Release: def _create_tag( self, tag: str, commit: str, tag_message: str = "" ) -> Iterator[None]: - tag_message = tag_message or "Release {tag}" + tag_message = tag_message or f"Release {tag}" # Create tag even in dry-run self.run(f"git tag -a -m '{tag_message}' '{tag}' {commit}") rollback_cmd = f"git tag -d '{tag}'"