From 99456d2fa630455ebf1a2422b5e827ec056f4130 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 14 Jun 2024 17:21:15 +0200 Subject: [PATCH 01/26] Fix unexpeced size of low cardinality column in functions --- src/Functions/IFunction.cpp | 101 ++++++++++++++++++++++++------------ 1 file changed, 68 insertions(+), 33 deletions(-) diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 31695fc95d5..9217071ca11 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -50,43 +50,78 @@ bool allArgumentsAreConstants(const ColumnsWithTypeAndName & args) ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( ColumnsWithTypeAndName & args, bool can_be_executed_on_default_arguments, size_t input_rows_count) { - size_t num_rows = input_rows_count; + /// We return the LC indexes so the LC can be reconstructed with the function result ColumnPtr indexes; - /// Find first LowCardinality column and replace it to nested dictionary. - for (auto & column : args) + size_t number_low_cardinality_columns = 0; + size_t last_low_cardinality = 0; + size_t number_const_columns = 0; + size_t number_full_columns = 0; + + for (size_t i = 0; i < args.size(); i++) { - if (const auto * low_cardinality_column = checkAndGetColumn(column.column.get())) + auto const & arg = args[i]; + if (checkAndGetColumn(arg.column.get())) { - /// Single LowCardinality column is supported now. - if (indexes) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected single dictionary argument for function."); - - const auto * low_cardinality_type = checkAndGetDataType(column.type.get()); - - if (!low_cardinality_type) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Incompatible type for LowCardinality column: {}", - column.type->getName()); - - if (can_be_executed_on_default_arguments) - { - /// Normal case, when function can be executed on values' default. - column.column = low_cardinality_column->getDictionary().getNestedColumn(); - indexes = low_cardinality_column->getIndexesPtr(); - } - else - { - /// Special case when default value can't be used. Example: 1 % LowCardinality(Int). - /// LowCardinality always contains default, so 1 % 0 will throw exception in normal case. - auto dict_encoded = low_cardinality_column->getMinimalDictionaryEncodedColumn(0, low_cardinality_column->size()); - column.column = dict_encoded.dictionary; - indexes = dict_encoded.indexes; - } - - num_rows = column.column->size(); - column.type = low_cardinality_type->getDictionaryType(); + number_low_cardinality_columns++; + last_low_cardinality = i; } + else if (checkAndGetColumn(arg.column.get())) + number_const_columns++; + else + number_full_columns++; + } + + if (!number_low_cardinality_columns && !number_const_columns) + return nullptr; + + if (number_full_columns > 0 || number_low_cardinality_columns > 1) + { + /// If there is a single full column, we can't replace the LC column with its dictionary, as it won't match + /// the size or order of the full columns. Same if there are 2 or more low cardinality columns + for (auto & arg : args) + { + if (const auto * column_lc = checkAndGetColumn(arg.column.get())) + { + arg.column = recursiveRemoveLowCardinality(arg.column); + chassert(arg.column->size() == input_rows_count); + + const auto * low_cardinality_type = checkAndGetDataType(arg.type.get()); + if (!low_cardinality_type) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incompatible type for LowCardinality column: {}", arg.type->getName()); + arg.type = recursiveRemoveLowCardinality(arg.type); + } + } + } + else if (number_low_cardinality_columns == 1) + { + auto & lc_arg = args[last_low_cardinality]; + + const auto * low_cardinality_type = checkAndGetDataType(lc_arg.type.get()); + if (!low_cardinality_type) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Incompatible type for LowCardinality column: {}", lc_arg.type->getName()); + + const auto * low_cardinality_column = checkAndGetColumn(lc_arg.column.get()); + chassert(low_cardinality_column); + + if (can_be_executed_on_default_arguments) + { + /// Normal case, when function can be executed on values' default. + lc_arg.column = low_cardinality_column->getDictionary().getNestedColumn(); + indexes = low_cardinality_column->getIndexesPtr(); + } + else + { + /// Special case when default value can't be used. Example: 1 % LowCardinality(Int). + /// LowCardinality always contains default, so 1 % 0 will throw exception in normal case. + auto dict_encoded = low_cardinality_column->getMinimalDictionaryEncodedColumn(0, low_cardinality_column->size()); + lc_arg.column = dict_encoded.dictionary; + indexes = dict_encoded.indexes; + } + + /// The new column will have a different number of rows, normally less but occasionally it might be more (NULL) + input_rows_count = lc_arg.column->size(); + lc_arg.type = low_cardinality_type->getDictionaryType(); } /// Change size of constants. @@ -94,7 +129,7 @@ ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( { if (const auto * column_const = checkAndGetColumn(column.column.get())) { - column.column = ColumnConst::create(recursiveRemoveLowCardinality(column_const->getDataColumnPtr()), num_rows); + column.column = ColumnConst::create(recursiveRemoveLowCardinality(column_const->getDataColumnPtr()), input_rows_count); column.type = recursiveRemoveLowCardinality(column.type); } } From 1c9df94a0a0b6fc9eb7ef302b47ac909c029e345 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 27 Jun 2024 21:26:39 +0200 Subject: [PATCH 02/26] Replace fix with LOGICAL_ERROR --- src/Functions/IFunction.cpp | 24 ++++++++---------------- 1 file changed, 8 insertions(+), 16 deletions(-) diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 9217071ca11..76ae8f33fbd 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -47,6 +47,9 @@ bool allArgumentsAreConstants(const ColumnsWithTypeAndName & args) return true; } +/// Replaces single low cardinality column in a function call by its dictionary +/// This can only happen after the arguments have been adapted in IFunctionOverloadResolver::getReturnType +/// as it's only possible if there is one low cardinality column and, optionally, const columns ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( ColumnsWithTypeAndName & args, bool can_be_executed_on_default_arguments, size_t input_rows_count) { @@ -77,21 +80,8 @@ ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( if (number_full_columns > 0 || number_low_cardinality_columns > 1) { - /// If there is a single full column, we can't replace the LC column with its dictionary, as it won't match - /// the size or order of the full columns. Same if there are 2 or more low cardinality columns - for (auto & arg : args) - { - if (const auto * column_lc = checkAndGetColumn(arg.column.get())) - { - arg.column = recursiveRemoveLowCardinality(arg.column); - chassert(arg.column->size() == input_rows_count); - - const auto * low_cardinality_type = checkAndGetDataType(arg.type.get()); - if (!low_cardinality_type) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Incompatible type for LowCardinality column: {}", arg.type->getName()); - arg.type = recursiveRemoveLowCardinality(arg.type); - } - } + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected low cardinality types found. Low cardinality: {}. Full {}. Const {}", + number_low_cardinality_columns, number_full_columns, number_const_columns); } else if (number_low_cardinality_columns == 1) { @@ -124,7 +114,7 @@ ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( lc_arg.type = low_cardinality_type->getDictionaryType(); } - /// Change size of constants. + /// Change size of constants for (auto & column : args) { if (const auto * column_const = checkAndGetColumn(column.column.get())) @@ -305,6 +295,8 @@ ColumnPtr IExecutableFunction::executeWithoutSparseColumns(const ColumnsWithType bool can_be_executed_on_default_arguments = canBeExecutedOnDefaultArguments(); const auto & dictionary_type = res_low_cardinality_type->getDictionaryType(); + /// The arguments should have been adapted in IFunctionOverloadResolver::getReturnType + /// So there is only one low cardinality column (and optionally some const columns) and no full column ColumnPtr indexes = replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( columns_without_low_cardinality, can_be_executed_on_default_arguments, input_rows_count); From c185d60375dd0fbc18d062197875ba463326f44a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 27 Jun 2024 21:59:14 +0200 Subject: [PATCH 03/26] Try an ugly fix --- src/Planner/PlannerActionsVisitor.cpp | 27 +++++++++++++++++++-------- 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 7a12d5d690d..13e96dc7016 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -485,16 +485,28 @@ public: return node; } - const ActionsDAG::Node * addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column) + [[nodiscard]] String addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column) { auto it = node_name_to_node.find(node_name); + if (it != node_name_to_node.end() && it->second->column) + return {node_name}; + if (it != node_name_to_node.end()) - return it->second; + { + /// There is a node with this name, but it doesn't have a column + /// This likely happens because we executed the query until WithMergeableState with a const node in the + /// WHERE clause. As the results of headers are materialized, the column was removed + /// Let's add a new column and keep this + String dupped_name{node_name + "_dupped"}; + const auto * node = &actions_dag.addColumn(column); + node_name_to_node[dupped_name] = node; + return dupped_name; + } const auto * node = &actions_dag.addColumn(column); node_name_to_node[node->result_name] = node; - return node; + return {node_name}; } template @@ -723,7 +735,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi column.type = constant_type; column.column = column.type->createColumnConst(1, constant_literal); - actions_stack[0].addConstantIfNecessary(constant_node_name, column); + String final_name = actions_stack[0].addConstantIfNecessary(constant_node_name, column); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) @@ -732,8 +744,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi actions_stack_node.addInputConstantColumnIfNecessary(constant_node_name, column); } - return {constant_node_name, Levels(0)}; - + return {final_name, Levels(0)}; } PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitLambda(const QueryTreeNodePtr & node) @@ -862,7 +873,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma else column.column = std::move(column_set); - actions_stack[0].addConstantIfNecessary(column.name, column); + String final_name = actions_stack[0].addConstantIfNecessary(column.name, column); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) @@ -871,7 +882,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma actions_stack_node.addInputConstantColumnIfNecessary(column.name, column); } - return {column.name, Levels(0)}; + return {final_name, Levels(0)}; } PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitIndexHintFunction(const QueryTreeNodePtr & node) From fed573ffee1bb57f0b9cff7f6f1c7d5a542af51c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 28 Jun 2024 17:12:10 +0200 Subject: [PATCH 04/26] Extend fix --- src/Planner/PlannerActionsVisitor.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 13e96dc7016..1c9553032c2 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -535,7 +535,7 @@ public: } private: - std::unordered_map node_name_to_node; + std::unordered_map node_name_to_node; ActionsDAG & actions_dag; QueryTreeNodePtr scope_node; }; @@ -741,7 +741,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi for (size_t i = 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputConstantColumnIfNecessary(constant_node_name, column); + actions_stack_node.addInputConstantColumnIfNecessary(final_name, column); } return {final_name, Levels(0)}; @@ -879,7 +879,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma for (size_t i = 1; i < actions_stack_size; ++i) { auto & actions_stack_node = actions_stack[i]; - actions_stack_node.addInputConstantColumnIfNecessary(column.name, column); + actions_stack_node.addInputConstantColumnIfNecessary(final_name, column); } return {final_name, Levels(0)}; From 74a5d56f1a404a7271b5a177e4902d0c319195d9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 1 Jul 2024 19:32:22 +0200 Subject: [PATCH 05/26] Fix interpolate and add tests --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 4 +- src/Functions/IFunction.cpp | 6 ++ src/Planner/Planner.cpp | 7 +- src/Planner/PlannerActionsVisitor.cpp | 36 ++++++--- src/Planner/PlannerActionsVisitor.h | 9 ++- ..._no_aggregates_and_constant_keys.reference | 4 +- ...nality_group_by_distributed_plan.reference | 55 +++++++++++++ ..._cardinality_group_by_distributed_plan.sql | 80 +++++++++++++++++++ 8 files changed, 185 insertions(+), 16 deletions(-) create mode 100644 tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.reference create mode 100644 tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.sql diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 8860050c5b9..165256479ce 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -4100,7 +4100,9 @@ void QueryAnalyzer::resolveInterpolateColumnsNodeList(QueryTreeNodePtr & interpo auto * column_to_interpolate = interpolate_node_typed.getExpression()->as(); if (!column_to_interpolate) - throw Exception(ErrorCodes::LOGICAL_ERROR, "INTERPOLATE can work only for indentifiers, but {} is found", + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "INTERPOLATE can work only for identifiers, but {} is found", interpolate_node_typed.getExpression()->formatASTForErrorMessage()); auto column_to_interpolate_name = column_to_interpolate->getIdentifier().getFullName(); diff --git a/src/Functions/IFunction.cpp b/src/Functions/IFunction.cpp index 76ae8f33fbd..8b092ba9b6e 100644 --- a/src/Functions/IFunction.cpp +++ b/src/Functions/IFunction.cpp @@ -80,8 +80,14 @@ ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes( if (number_full_columns > 0 || number_low_cardinality_columns > 1) { + /// This should not be possible but currently there are multiple tests in CI failing because of it + /// TODO: Fix those cases, then enable this exception +#if 0 throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected low cardinality types found. Low cardinality: {}. Full {}. Const {}", number_low_cardinality_columns, number_full_columns, number_const_columns); +#else + return nullptr; +#endif } else if (number_low_cardinality_columns == 1) { diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 2d42ed73223..93a4ea01ff0 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -742,7 +742,12 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, { auto & interpolate_node_typed = interpolate_node->as(); - PlannerActionsVisitor planner_actions_visitor(planner_context); + PlannerActionsVisitor planner_actions_visitor( + planner_context, + /* use_column_identifier_as_action_node_name_, (default value)*/ true, + /// Prefer the INPUT to CONSTANT nodes (actions must be non constant) + /* prefer_const_column_to_input */ false); + auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag, interpolate_node_typed.getExpression()); if (expression_to_interpolate_expression_nodes.size() != 1) diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index 1c9553032c2..a199420c9bd 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -485,19 +485,24 @@ public: return node; } - [[nodiscard]] String addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column) + [[nodiscard]] String + addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column, bool prefer_const_column_to_input) { + chassert(column.column != nullptr); auto it = node_name_to_node.find(node_name); - if (it != node_name_to_node.end() && it->second->column) + if (it != node_name_to_node.end() && (!prefer_const_column_to_input || it->second->column)) return {node_name}; if (it != node_name_to_node.end()) { /// There is a node with this name, but it doesn't have a column /// This likely happens because we executed the query until WithMergeableState with a const node in the - /// WHERE clause. As the results of headers are materialized, the column was removed + /// WHERE clause and, as the results of headers are materialized, the column was removed /// Let's add a new column and keep this String dupped_name{node_name + "_dupped"}; + if (node_name_to_node.find(dupped_name) != node_name_to_node.end()) + return dupped_name; + const auto * node = &actions_dag.addColumn(column); node_name_to_node[dupped_name] = node; return dupped_name; @@ -543,9 +548,11 @@ private: class PlannerActionsVisitorImpl { public: - PlannerActionsVisitorImpl(ActionsDAG & actions_dag, + PlannerActionsVisitorImpl( + ActionsDAG & actions_dag, const PlannerContextPtr & planner_context_, - bool use_column_identifier_as_action_node_name_); + bool use_column_identifier_as_action_node_name_, + bool prefer_const_column_to_input_); ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node); @@ -605,14 +612,18 @@ private: const PlannerContextPtr planner_context; ActionNodeNameHelper action_node_name_helper; bool use_column_identifier_as_action_node_name; + bool prefer_const_column_to_input; }; -PlannerActionsVisitorImpl::PlannerActionsVisitorImpl(ActionsDAG & actions_dag, +PlannerActionsVisitorImpl::PlannerActionsVisitorImpl( + ActionsDAG & actions_dag, const PlannerContextPtr & planner_context_, - bool use_column_identifier_as_action_node_name_) + bool use_column_identifier_as_action_node_name_, + bool prefer_const_column_to_input_) : planner_context(planner_context_) , action_node_name_helper(node_to_node_name, *planner_context, use_column_identifier_as_action_node_name_) , use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_) + , prefer_const_column_to_input(prefer_const_column_to_input_) { actions_stack.emplace_back(actions_dag, nullptr); } @@ -735,7 +746,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi column.type = constant_type; column.column = column.type->createColumnConst(1, constant_literal); - String final_name = actions_stack[0].addConstantIfNecessary(constant_node_name, column); + String final_name = actions_stack[0].addConstantIfNecessary(constant_node_name, column, prefer_const_column_to_input); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) @@ -873,7 +884,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma else column.column = std::move(column_set); - String final_name = actions_stack[0].addConstantIfNecessary(column.name, column); + String final_name = actions_stack[0].addConstantIfNecessary(column.name, column, prefer_const_column_to_input); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) @@ -1019,14 +1030,17 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi } -PlannerActionsVisitor::PlannerActionsVisitor(const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_) +PlannerActionsVisitor::PlannerActionsVisitor( + const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_, bool prefer_const_column_to_input_) : planner_context(planner_context_) , use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_) + , prefer_const_column_to_input(prefer_const_column_to_input_) {} ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAG & actions_dag, QueryTreeNodePtr expression_node) { - PlannerActionsVisitorImpl actions_visitor_impl(actions_dag, planner_context, use_column_identifier_as_action_node_name); + PlannerActionsVisitorImpl actions_visitor_impl( + actions_dag, planner_context, use_column_identifier_as_action_node_name, prefer_const_column_to_input); return actions_visitor_impl.visit(expression_node); } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 6bb32047327..4bec2d2bb8a 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -27,11 +27,17 @@ using PlannerContextPtr = std::shared_ptr; * During actions build, there is special handling for following functions: * 1. Aggregate functions are added in actions dag as INPUT nodes. Aggregate functions arguments are not added. * 2. For function `in` and its variants, already collected sets from planner context are used. + * 3. When building actions that use CONSTANT nodes, by default we ignore pre-existing INPUTs if those don't have + * a column (a const column always has a column). This is for compatibility with previous headers. We disable this + * behaviour when we explicitly want to override CONSTANT nodes with the input (resolving InterpolateNode for example) */ class PlannerActionsVisitor { public: - explicit PlannerActionsVisitor(const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_ = true); + explicit PlannerActionsVisitor( + const PlannerContextPtr & planner_context_, + bool use_column_identifier_as_action_node_name_ = true, + bool prefer_const_column_to_input_ = true); /** Add actions necessary to calculate expression node into expression dag. * Necessary actions are not added in actions dag output. @@ -42,6 +48,7 @@ public: private: const PlannerContextPtr planner_context; bool use_column_identifier_as_action_node_name = true; + bool prefer_const_column_to_input = true; }; /** Calculate query tree expression node action dag name and add them into node to name map. diff --git a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference index 63b8a9d14fc..fc77ed8a241 100644 --- a/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference +++ b/tests/queries/0_stateless/00257_shard_no_aggregates_and_constant_keys.reference @@ -8,13 +8,13 @@ 40 41 -0 +41 2 42 2 42 43 -0 +43 11 11 diff --git a/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.reference b/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.reference new file mode 100644 index 00000000000..1508c24f410 --- /dev/null +++ b/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.reference @@ -0,0 +1,55 @@ +-- { echoOn } +SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8))) +FROM system.one +GROUP BY '666'; +6.666.8 +SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8))) +FROM remote('127.0.0.{1,1}', 'system.one') +GROUP BY '666'; +6.666.8 +-- https://github.com/ClickHouse/ClickHouse/issues/63006 +SELECT + 6, + concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a, + concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b +FROM system.one +GROUP BY toNullable(6) + WITH ROLLUP +WITH TOTALS; +6 World666666 \N +6 World666666 \N + +6 World666666 \N +SELECT + 6, + concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a, + concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b +FROM remote('127.0.0.1') +GROUP BY toNullable(6) + WITH ROLLUP + WITH TOTALS; +6 World666666 \N +6 World666666 \N + +6 World666666 \N +-- { echoOn } +SELECT + '%', + tuple(concat('%', 1, toLowCardinality(toLowCardinality(toNullable(materialize(1)))), currentDatabase(), 101., toNullable(13), '%AS%id_02%', toNullable(toNullable(10)), toLowCardinality(toNullable(10)), 10, 10)), + (toDecimal128(99.67, 6), 36, 61, 14) +FROM dist_03174 +WHERE dummy IN (0, '255') +GROUP BY + toNullable(13), + (99.67, 61, toLowCardinality(14)); +% ('%11default10113%AS%id_02%10101010') (99.67,36,61,14) +-- { echoOn } +SELECT + 38, + concat(position(concat(concat(position(concat(toUInt256(3)), 'ca', 2), 3), NULLIF(1, materialize(toLowCardinality(1)))), toLowCardinality(toNullable('ca'))), concat(NULLIF(1, 1), concat(3), toNullable(3))) +FROM set_index_not__fuzz_0 +GROUP BY + toNullable(3), + concat(concat(CAST(NULL, 'Nullable(Int8)'), toNullable(3))) +FORMAT Null +SETTINGS max_threads = 1, allow_experimental_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 3, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1; diff --git a/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.sql b/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.sql new file mode 100644 index 00000000000..d397d30e285 --- /dev/null +++ b/tests/queries/0_stateless/03174_low_cardinality_group_by_distributed_plan.sql @@ -0,0 +1,80 @@ +-- There are various tests that check that group by keys don't propagate into functions replacing const arguments +-- by full (empty) columns + +DROP TABLE IF EXISTS dist_03174; +DROP TABLE IF EXISTS set_index_not__fuzz_0; + +-- https://github.com/ClickHouse/ClickHouse/issues/63006 + +SET allow_experimental_analyzer=1; + +-- { echoOn } +SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8))) +FROM system.one +GROUP BY '666'; + +SELECT concatWithSeparator('.', toUInt128(6), '666' as b, materialize(toLowCardinality(8))) +FROM remote('127.0.0.{1,1}', 'system.one') +GROUP BY '666'; + +-- https://github.com/ClickHouse/ClickHouse/issues/63006 +SELECT + 6, + concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a, + concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b +FROM system.one +GROUP BY toNullable(6) + WITH ROLLUP +WITH TOTALS; + +SELECT + 6, + concat(' World', toUInt128(6), 6, 6, 6, toNullable(6), materialize(toLowCardinality(toNullable(toUInt128(6))))) AS a, + concat(concat(' World', 6, toLowCardinality(6), ' World', toUInt256(6), materialize(6), 6, toNullable(6), 6, 6, NULL, 6, 6), ' World', 6, 6, 6, 6, toUInt256(6), NULL, 6, 6) AS b +FROM remote('127.0.0.1') +GROUP BY toNullable(6) + WITH ROLLUP + WITH TOTALS; + +-- https://github.com/ClickHouse/ClickHouse/issues/64945 +-- { echoOff } +CREATE TABLE dist_03174 AS system.one ENGINE = Distributed(test_cluster_two_shards, system, one, dummy); + +-- { echoOn } +SELECT + '%', + tuple(concat('%', 1, toLowCardinality(toLowCardinality(toNullable(materialize(1)))), currentDatabase(), 101., toNullable(13), '%AS%id_02%', toNullable(toNullable(10)), toLowCardinality(toNullable(10)), 10, 10)), + (toDecimal128(99.67, 6), 36, 61, 14) +FROM dist_03174 +WHERE dummy IN (0, '255') +GROUP BY + toNullable(13), + (99.67, 61, toLowCardinality(14)); + +-- Parallel replicas +-- { echoOff } +CREATE TABLE set_index_not__fuzz_0 +( + `name` String, + `status` Enum8('alive' = 0, 'rip' = 1), + INDEX idx_status status TYPE set(2) GRANULARITY 1 +) +ENGINE = MergeTree() +ORDER BY name; + +INSERT INTO set_index_not__fuzz_0 SELECT * FROM generateRandom() LIMIT 10; + +-- { echoOn } +SELECT + 38, + concat(position(concat(concat(position(concat(toUInt256(3)), 'ca', 2), 3), NULLIF(1, materialize(toLowCardinality(1)))), toLowCardinality(toNullable('ca'))), concat(NULLIF(1, 1), concat(3), toNullable(3))) +FROM set_index_not__fuzz_0 +GROUP BY + toNullable(3), + concat(concat(CAST(NULL, 'Nullable(Int8)'), toNullable(3))) +FORMAT Null +SETTINGS max_threads = 1, allow_experimental_analyzer = 1, cluster_for_parallel_replicas = 'parallel_replicas', max_parallel_replicas = 3, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_threads = 1; + +-- { echoOff } +DROP TABLE IF EXISTS dist_03174; +DROP TABLE IF EXISTS set_index_not__fuzz_0; From 0e559ff7b94c55fe35d0db7174e523180f46e998 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 2 Jul 2024 11:50:43 +0200 Subject: [PATCH 06/26] Better name for flag --- src/Planner/Planner.cpp | 2 +- src/Planner/PlannerActionsVisitor.cpp | 26 ++++++++++++++------------ src/Planner/PlannerActionsVisitor.h | 4 ++-- 3 files changed, 17 insertions(+), 15 deletions(-) diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 93a4ea01ff0..260fbabf26a 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -746,7 +746,7 @@ void addWithFillStepIfNeeded(QueryPlan & query_plan, planner_context, /* use_column_identifier_as_action_node_name_, (default value)*/ true, /// Prefer the INPUT to CONSTANT nodes (actions must be non constant) - /* prefer_const_column_to_input */ false); + /* always_use_const_column_for_constant_nodes */ false); auto expression_to_interpolate_expression_nodes = planner_actions_visitor.visit(*interpolate_actions_dag, interpolate_node_typed.getExpression()); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index a199420c9bd..e9e1f4edcc2 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -485,12 +485,12 @@ public: return node; } - [[nodiscard]] String - addConstantIfNecessary(const std::string & node_name, const ColumnWithTypeAndName & column, bool prefer_const_column_to_input) + [[nodiscard]] String addConstantIfNecessary( + const std::string & node_name, const ColumnWithTypeAndName & column, bool always_use_const_column_for_constant_nodes) { chassert(column.column != nullptr); auto it = node_name_to_node.find(node_name); - if (it != node_name_to_node.end() && (!prefer_const_column_to_input || it->second->column)) + if (it != node_name_to_node.end() && (!always_use_const_column_for_constant_nodes || it->second->column)) return {node_name}; if (it != node_name_to_node.end()) @@ -552,7 +552,7 @@ public: ActionsDAG & actions_dag, const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_, - bool prefer_const_column_to_input_); + bool always_use_const_column_for_constant_nodes_); ActionsDAG::NodeRawConstPtrs visit(QueryTreeNodePtr expression_node); @@ -612,18 +612,18 @@ private: const PlannerContextPtr planner_context; ActionNodeNameHelper action_node_name_helper; bool use_column_identifier_as_action_node_name; - bool prefer_const_column_to_input; + bool always_use_const_column_for_constant_nodes; }; PlannerActionsVisitorImpl::PlannerActionsVisitorImpl( ActionsDAG & actions_dag, const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_, - bool prefer_const_column_to_input_) + bool always_use_const_column_for_constant_nodes_) : planner_context(planner_context_) , action_node_name_helper(node_to_node_name, *planner_context, use_column_identifier_as_action_node_name_) , use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_) - , prefer_const_column_to_input(prefer_const_column_to_input_) + , always_use_const_column_for_constant_nodes(always_use_const_column_for_constant_nodes_) { actions_stack.emplace_back(actions_dag, nullptr); } @@ -746,7 +746,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi column.type = constant_type; column.column = column.type->createColumnConst(1, constant_literal); - String final_name = actions_stack[0].addConstantIfNecessary(constant_node_name, column, prefer_const_column_to_input); + String final_name = actions_stack[0].addConstantIfNecessary(constant_node_name, column, always_use_const_column_for_constant_nodes); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) @@ -884,7 +884,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::ma else column.column = std::move(column_set); - String final_name = actions_stack[0].addConstantIfNecessary(column.name, column, prefer_const_column_to_input); + String final_name = actions_stack[0].addConstantIfNecessary(column.name, column, always_use_const_column_for_constant_nodes); size_t actions_stack_size = actions_stack.size(); for (size_t i = 1; i < actions_stack_size; ++i) @@ -1031,16 +1031,18 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi } PlannerActionsVisitor::PlannerActionsVisitor( - const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_, bool prefer_const_column_to_input_) + const PlannerContextPtr & planner_context_, + bool use_column_identifier_as_action_node_name_, + bool always_use_const_column_for_constant_nodes_) : planner_context(planner_context_) , use_column_identifier_as_action_node_name(use_column_identifier_as_action_node_name_) - , prefer_const_column_to_input(prefer_const_column_to_input_) + , always_use_const_column_for_constant_nodes(always_use_const_column_for_constant_nodes_) {} ActionsDAG::NodeRawConstPtrs PlannerActionsVisitor::visit(ActionsDAG & actions_dag, QueryTreeNodePtr expression_node) { PlannerActionsVisitorImpl actions_visitor_impl( - actions_dag, planner_context, use_column_identifier_as_action_node_name, prefer_const_column_to_input); + actions_dag, planner_context, use_column_identifier_as_action_node_name, always_use_const_column_for_constant_nodes); return actions_visitor_impl.visit(expression_node); } diff --git a/src/Planner/PlannerActionsVisitor.h b/src/Planner/PlannerActionsVisitor.h index 4bec2d2bb8a..1dbd149bc4b 100644 --- a/src/Planner/PlannerActionsVisitor.h +++ b/src/Planner/PlannerActionsVisitor.h @@ -37,7 +37,7 @@ public: explicit PlannerActionsVisitor( const PlannerContextPtr & planner_context_, bool use_column_identifier_as_action_node_name_ = true, - bool prefer_const_column_to_input_ = true); + bool always_use_const_column_for_constant_nodes_ = true); /** Add actions necessary to calculate expression node into expression dag. * Necessary actions are not added in actions dag output. @@ -48,7 +48,7 @@ public: private: const PlannerContextPtr planner_context; bool use_column_identifier_as_action_node_name = true; - bool prefer_const_column_to_input = true; + bool always_use_const_column_for_constant_nodes = true; }; /** Calculate query tree expression node action dag name and add them into node to name map. From 67f5ffc5920892b34b1c8b7176337c39cafe7141 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 16 Jul 2024 20:02:53 +0000 Subject: [PATCH 07/26] Fixups --- tests/performance/final_big_column.xml | 4 ++-- tests/performance/function_tokens.xml | 4 ++-- tests/performance/polymorphic_parts_s.xml | 8 ++++---- tests/performance/scripts/report.py | 2 +- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/performance/final_big_column.xml b/tests/performance/final_big_column.xml index 1fd586d2d90..5225b3d7ad4 100644 --- a/tests/performance/final_big_column.xml +++ b/tests/performance/final_big_column.xml @@ -10,8 +10,8 @@ PARTITION BY toYYYYMM(d) ORDER BY key - INSERT INTO optimized_select_final SELECT toDate('2000-01-01'), 2*number, randomPrintableASCII(1000) FROM numbers(5000000) - INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), 2*number+1, randomPrintableASCII(1000) FROM numbers(5000000) + INSERT INTO optimized_select_final SELECT toDate('2000-01-01'), 2*number, randomPrintableASCII(1000) FROM numbers(2500000) + INSERT INTO optimized_select_final SELECT toDate('2020-01-01'), 2*number+1, randomPrintableASCII(1000) FROM numbers(2500000) SELECT * FROM optimized_select_final FINAL FORMAT Null SETTINGS max_threads = 8 SELECT * FROM optimized_select_final FINAL WHERE key % 10 = 0 FORMAT Null diff --git a/tests/performance/function_tokens.xml b/tests/performance/function_tokens.xml index 1ff56323d62..bc2bc71a933 100644 --- a/tests/performance/function_tokens.xml +++ b/tests/performance/function_tokens.xml @@ -1,5 +1,5 @@ with 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' as s select splitByChar(' ', materialize(s)) as w from numbers(1000000) - with 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' as s select splitByRegexp(' ', materialize(s)) as w from numbers(1000000) - with 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' as s select splitByRegexp('\s+', materialize(s)) as w from numbers(100000) + with 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' as s select splitByRegexp(' ', materialize(s)) as w from numbers(200000) + with 'Many years later as he faced the firing squad, Colonel Aureliano Buendia was to remember that distant afternoon when his father took him to discover ice.' as s select splitByRegexp('\s+', materialize(s)) as w from numbers(20000) diff --git a/tests/performance/polymorphic_parts_s.xml b/tests/performance/polymorphic_parts_s.xml index b4dd87a7ae3..5fe1ffffe1d 100644 --- a/tests/performance/polymorphic_parts_s.xml +++ b/tests/performance/polymorphic_parts_s.xml @@ -24,10 +24,10 @@ 1 - - INSERT INTO hits_wide(UserID) SELECT rand() FROM numbers(100) - INSERT INTO hits_compact(UserID) SELECT rand() FROM numbers(1000) - INSERT INTO hits_buffer(UserID) SELECT rand() FROM numbers(100) + + INSERT INTO hits_wide(UserID) SELECT rand() FROM numbers(50) + INSERT INTO hits_compact(UserID) SELECT rand() FROM numbers(500) + INSERT INTO hits_buffer(UserID) SELECT rand() FROM numbers(50) DROP TABLE IF EXISTS hits_wide DROP TABLE IF EXISTS hits_compact diff --git a/tests/performance/scripts/report.py b/tests/performance/scripts/report.py index c2bc773bd54..e45d709ca2c 100755 --- a/tests/performance/scripts/report.py +++ b/tests/performance/scripts/report.py @@ -555,7 +555,7 @@ if args.report == "main": "Total client time for measured query runs, s", # 2 "Queries", # 3 "Longest query, total for measured runs, s", # 4 - "Wall clock time per query, s", # 5 + "Average query wall clock time, s", # 5 "Shortest query, total for measured runs, s", # 6 "", # Runs #7 ] From e3b2fbf7ec1a322fa4fcbd808a3a19dd6ec3b8ee Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 16 Jul 2024 15:37:50 +0200 Subject: [PATCH 08/26] CI: POC for Auto Releases --- .github/workflows/auto_release.yml | 42 ++++--- .github/workflows/create_release.yml | 9 +- tests/ci/auto_release.py | 169 +++++++++++++-------------- tests/ci/ci_utils.py | 6 +- 4 files changed, 121 insertions(+), 105 deletions(-) diff --git a/.github/workflows/auto_release.yml b/.github/workflows/auto_release.yml index f1a6b307b40..e90b183b8d6 100644 --- a/.github/workflows/auto_release.yml +++ b/.github/workflows/auto_release.yml @@ -1,44 +1,58 @@ name: AutoRelease env: - # Force the stdout and stderr streams to be unbuffered PYTHONUNBUFFERED: 1 concurrency: - group: auto-release + group: release on: # yamllint disable-line rule:truthy # schedule: # - cron: '0 10-16 * * 1-5' workflow_dispatch: jobs: - CherryPick: - runs-on: [self-hosted, style-checker-aarch64] + AutoRelease: + runs-on: [self-hosted, release-maker] steps: + - name: DebugInfo + uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6 - name: Set envs - # https://docs.github.com/en/actions/learn-github-actions/workflow-commands-for-github-actions#multiline-strings run: | cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/cherry_pick ROBOT_CLICKHOUSE_SSH_KEY<> "$GITHUB_ENV" + - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0].release_branch }} + if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0] }} + uses: ./.github/workflows/create_release.yml + with: + type: patch + ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0].commit_sha }} + dry-run: true + autorelease: true + - name: Post Slack Message + if: ${{ !cancelled() }} + run: | + echo Slack Message + - name: Clean up run: | docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index d8d27531f28..96cd46f583a 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -22,6 +22,10 @@ concurrency: required: false default: true type: boolean + autorelease: + required: false + default: false + type: boolean jobs: CreateRelease: @@ -30,8 +34,10 @@ jobs: runs-on: [self-hosted, release-maker] steps: - name: DebugInfo + if: ${{ ! inputs.autorelease }} uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6 - name: Set envs + if: ${{ ! inputs.autorelease }} # https://docs.github.com/en/actions/learn-github-actions/workflow-commands-for-github-actions#multiline-strings run: | cat >> "$GITHUB_ENV" << 'EOF' @@ -41,6 +47,7 @@ jobs: RELEASE_INFO_FILE=${{ runner.temp }}/release_info.json EOF - name: Check out repository code + if: ${{ ! inputs.autorelease }} uses: ClickHouse/checkout@v1 with: token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} @@ -161,6 +168,6 @@ jobs: export CHECK_NAME="Docker keeper image" python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} - name: Post Slack Message - if: always() + if: ${{ !cancelled() }} run: | echo Slack Message diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index bfd19b11e6d..88e91eb8aed 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -1,13 +1,16 @@ import argparse -from datetime import timedelta, datetime +import dataclasses +import json import logging import os -from commit_status_helper import get_commit_filtered_statuses +from typing import List + from get_robot_token import get_best_robot_token from github_helper import GitHub -from release import Release, Repo as ReleaseRepo, RELEASE_READY_STATUS -from report import SUCCESS from ssh import SSHKey +from ci_utils import Shell +from env_helper import GITHUB_REPOSITORY +from report import SUCCESS LOGGER_NAME = __name__ HELPER_LOGGERS = ["github_helper", LOGGER_NAME] @@ -20,116 +23,104 @@ def parse_args(): "branches and do a release in case for green builds." ) parser.add_argument("--token", help="GitHub token, if not set, used from smm") - parser.add_argument( - "--repo", default="ClickHouse/ClickHouse", help="Repo owner/name" - ) - parser.add_argument("--dry-run", action="store_true", help="Do not create anything") - parser.add_argument( - "--release-after-days", - type=int, - default=3, - help="Do automatic release on the latest green commit after the latest " - "release if the newest release is older than the specified days", - ) - parser.add_argument( - "--debug-helpers", - action="store_true", - help="Add debug logging for this script and github_helper", - ) - parser.add_argument( - "--remote-protocol", - "-p", - default="ssh", - choices=ReleaseRepo.VALID, - help="repo protocol for git commands remote, 'origin' is a special case and " - "uses 'origin' as a remote", - ) return parser.parse_args() +MAX_NUMBER_OF_COMMITS_TO_CONSIDER_FOR_RELEASE = 5 +AUTORELEASE_INFO_FILE = "/tmp/autorelease_info.json" + + +@dataclasses.dataclass +class ReleaseParams: + release_branch: str + commit_sha: str + + +@dataclasses.dataclass +class AutoReleaseInfo: + releases: List[ReleaseParams] + + def add_release(self, release_params: ReleaseParams): + self.releases.append(release_params) + + def dump(self): + print(f"Dump release info into [{AUTORELEASE_INFO_FILE}]") + with open(AUTORELEASE_INFO_FILE, "w", encoding="utf-8") as f: + print(json.dumps(dataclasses.asdict(self), indent=2), file=f) + + def main(): args = parse_args() - logging.basicConfig(level=logging.INFO) - if args.debug_helpers: - for logger_name in HELPER_LOGGERS: - logging.getLogger(logger_name).setLevel(logging.DEBUG) token = args.token or get_best_robot_token() - days_as_timedelta = timedelta(days=args.release_after_days) - now = datetime.now() - + assert len(token) > 10 + os.environ["GH_TOKEN"] = token + (Shell.run("gh auth status", check=True)) gh = GitHub(token) - prs = gh.get_release_pulls(args.repo) + prs = gh.get_release_pulls(GITHUB_REPOSITORY) branch_names = [pr.head.ref for pr in prs] - logger.info("Found release branches: %s\n ", " \n".join(branch_names)) - repo = gh.get_repo(args.repo) + print(f"Found release branches [{branch_names}]") + repo = gh.get_repo(GITHUB_REPOSITORY) - # In general there is no guarantee on which order the refs/commits are - # returned from the API, so we have to order them. + autoRelease_info = AutoReleaseInfo(releases=[]) for pr in prs: - logger.info("Checking PR %s", pr.head.ref) + print(f"Checking PR [{pr.head.ref}]") refs = list(repo.get_git_matching_refs(f"tags/v{pr.head.ref}")) refs.sort(key=lambda ref: ref.ref) latest_release_tag_ref = refs[-1] latest_release_tag = repo.get_git_tag(latest_release_tag_ref.object.sha) - logger.info("That last release was done at %s", latest_release_tag.tagger.date) - - if latest_release_tag.tagger.date + days_as_timedelta > now: - logger.info( - "Not enough days since the last release %s," - " no automatic release can be done", - latest_release_tag.tag, + commit_num = int( + Shell.run( + f"git rev-list --count {latest_release_tag.tag}..origin/{pr.head.ref}", + check=True, ) - continue - - unreleased_commits = list( - repo.get_commits(sha=pr.head.ref, since=latest_release_tag.tagger.date) ) - unreleased_commits.sort( - key=lambda commit: commit.commit.committer.date, reverse=True + print( + f"Previous release is [{latest_release_tag}] was [{commit_num}] commits before, date [{latest_release_tag.tagger.date}]" ) - - for commit in unreleased_commits: - logger.info("Checking statuses of commit %s", commit.sha) - statuses = get_commit_filtered_statuses(commit) - all_success = all(st.state == SUCCESS for st in statuses) - passed_ready_for_release_check = any( - st.context == RELEASE_READY_STATUS and st.state == SUCCESS - for st in statuses + commit_reverse_index = 0 + commit_found = False + commit_checked = False + commit_sha = "" + while ( + commit_reverse_index < commit_num - 1 + and commit_reverse_index < MAX_NUMBER_OF_COMMITS_TO_CONSIDER_FOR_RELEASE + ): + commit_checked = True + commit_sha = Shell.run( + f"git rev-list --max-count=1 --skip={commit_reverse_index} origin/{pr.head.ref}", + check=True, ) - if not (all_success and passed_ready_for_release_check): - logger.info("Commit is not green, thus not suitable for release") - continue - - logger.info("Commit is ready for release, let's release!") - - release = Release( - ReleaseRepo(args.repo, args.remote_protocol), - commit.sha, - "patch", - args.dry_run, - True, + print( + f"Check if commit [{commit_sha}] [{pr.head.ref}~{commit_reverse_index}] is ready for release" ) - try: - release.do(True, True, True) - except: - if release.has_rollback: - logging.error( - "!!The release process finished with error, read the output carefully!!" - ) - logging.error( - "Probably, rollback finished with error. " - "If you don't see any of the following commands in the output, " - "execute them manually:" - ) - release.log_rollback() - raise - logging.info("New release is done!") + commit_reverse_index += 1 + + cmd = f"gh api -H 'Accept: application/vnd.github.v3+json' /repos/{GITHUB_REPOSITORY}/commits/{commit_sha}/status" + ci_status_json = Shell.run(cmd, check=True) + ci_status = json.loads(ci_status_json)["state"] + if ci_status == SUCCESS: + commit_found = True break + if commit_found: + print( + f"Add release ready info for commit [{commit_sha}] and release branch [{pr.head.ref}]" + ) + autoRelease_info.add_release( + ReleaseParams(release_branch=pr.head.ref, commit_sha=commit_sha) + ) + else: + print(f"WARNING: No good commits found for release branch [{pr.head.ref}]") + if commit_checked: + print( + f"ERROR: CI is failed. check CI status for branch [{pr.head.ref}]" + ) + + autoRelease_info.dump() if __name__ == "__main__": diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 49f0447b5ca..0653374356f 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -95,7 +95,8 @@ class Shell: return res.stdout.strip() @classmethod - def run(cls, command): + def run(cls, command, check=False): + print(f"Run command [{command}]") res = "" result = subprocess.run( command, @@ -107,6 +108,9 @@ class Shell: ) if result.returncode == 0: res = result.stdout + elif check: + print(f"ERROR: stdout {result.stdout}, stderr {result.stderr}") + assert result.returncode == 0 return res.strip() @classmethod From 3de472cedc22c1b109256507d81a491f9cf57d58 Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 16 Jul 2024 17:07:49 +0200 Subject: [PATCH 09/26] add buddy, move release into action.yml try callable wf fix add ci buddy --- .github/actions/release/action.yml | 165 ++++++++++++++++++++++++ .github/workflows/auto_release.yml | 55 +++++++- .github/workflows/create_release.yml | 140 +------------------- tests/ci/artifactory.py | 32 ++--- tests/ci/auto_release.py | 183 ++++++++++++++++++--------- tests/ci/ci.py | 2 +- tests/ci/ci_buddy.py | 56 +++++++- tests/ci/ci_config.py | 3 + tests/ci/ci_utils.py | 76 ++++++++++- tests/ci/create_release.py | 106 ++++++++++------ tests/ci/pr_info.py | 13 +- 11 files changed, 560 insertions(+), 271 deletions(-) create mode 100644 .github/actions/release/action.yml diff --git a/.github/actions/release/action.yml b/.github/actions/release/action.yml new file mode 100644 index 00000000000..fd4436f1f16 --- /dev/null +++ b/.github/actions/release/action.yml @@ -0,0 +1,165 @@ +name: Release + +description: Makes patch releases and creates new release branch + +inputs: + ref: + description: 'Git reference (branch or commit sha) from which to create the release' + required: true + type: string + type: + description: 'The type of release: "new" for a new release or "patch" for a patch release' + required: true + type: choice + options: + - patch + - new + dry-run: + description: 'Dry run' + required: false + default: true + type: boolean + token: + required: true + type: string + +runs: + using: "composite" + steps: + - name: Prepare Release Info + shell: bash + run: | + python3 ./tests/ci/create_release.py --prepare-release-info \ + --ref ${{ inputs.ref }} --release-type ${{ inputs.type }} \ + ${{ inputs.dry-run && '--dry-run' || '' }} + echo "::group::Release Info" + python3 -m json.tool /tmp/release_info.json + echo "::endgroup::" + release_tag=$(jq -r '.release_tag' /tmp/release_info.json) + commit_sha=$(jq -r '.commit_sha' /tmp/release_info.json) + echo "Release Tag: $release_tag" + echo "RELEASE_TAG=$release_tag" >> "$GITHUB_ENV" + echo "COMMIT_SHA=$commit_sha" >> "$GITHUB_ENV" + - name: Download All Release Artifacts + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --download-packages ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Push Git Tag for the Release + shell: bash + run: | + python3 ./tests/ci/create_release.py --push-release-tag ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Push New Release Branch + if: ${{ inputs.type == 'new' }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --push-new-release-branch ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Bump CH Version and Update Contributors' List + shell: bash + run: | + python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Checkout master + shell: bash + run: | + git checkout master + - name: Bump Docker versions, Changelog, Security + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + [ "$(git branch --show-current)" != "master" ] && echo "not on the master" && exit 1 + echo "List versions" + ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv + echo "Update docker version" + ./utils/list-versions/update-docker-version.sh + echo "Generate ChangeLog" + export CI=1 + docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ + --volume=".:/ClickHouse" clickhouse/style-test \ + /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ + --gh-user-or-token=${{ inputs.token }} --jobs=5 \ + --output="/ClickHouse/docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} + git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md + echo "Generate Security" + python3 ./utils/security-generator/generate_security.py > SECURITY.md + git diff HEAD + - name: Create ChangeLog PR + if: ${{ inputs.type == 'patch' && ! inputs.dry-run }} + uses: peter-evans/create-pull-request@v6 + with: + author: "robot-clickhouse " + token: ${{ inputs.token }} + committer: "robot-clickhouse " + commit-message: Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} + branch: auto/${{ env.RELEASE_TAG }} + assignees: ${{ github.event.sender.login }} # assign the PR to the tag pusher + delete-branch: true + title: Update version_date.tsv and changelog after ${{ env.RELEASE_TAG }} + labels: do not test + body: | + Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} + ### Changelog category (leave one): + - Not for changelog (changelog entry is not required) + - name: Reset changes if Dry-run + if: ${{ inputs.dry-run }} + shell: bash + run: | + git reset --hard HEAD + - name: Checkout back to GITHUB_REF + shell: bash + run: | + git checkout "$GITHUB_REF_NAME" + - name: Create GH Release + shell: bash + if: ${{ inputs.type == 'patch' }} + run: | + python3 ./tests/ci/create_release.py --create-gh-release \ + ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Export TGZ Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --export-tgz ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Test TGZ Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --test-tgz ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Export RPM Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --export-rpm ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Test RPM Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --test-rpm ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Export Debian Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --export-debian ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Test Debian Packages + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + python3 ./tests/ci/artifactory.py --test-debian ${{ inputs.dry-run && '--dry-run' || '' }} + - name: Docker clickhouse/clickhouse-server building + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + cd "./tests/ci" + export CHECK_NAME="Docker server image" + python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + - name: Docker clickhouse/clickhouse-keeper building + if: ${{ inputs.type == 'patch' }} + shell: bash + run: | + cd "./tests/ci" + export CHECK_NAME="Docker keeper image" + python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + - name: Post Slack Message + if: ${{ !cancelled() }} + shell: bash + run: | + python3 ./tests/ci/create_release.py --post-status ${{ inputs.dry-run && '--dry-run' || '' }} diff --git a/.github/workflows/auto_release.yml b/.github/workflows/auto_release.yml index e90b183b8d6..e0a94d3bbb1 100644 --- a/.github/workflows/auto_release.yml +++ b/.github/workflows/auto_release.yml @@ -2,6 +2,7 @@ name: AutoRelease env: PYTHONUNBUFFERED: 1 + DRY_RUN: true concurrency: group: release @@ -9,6 +10,12 @@ on: # yamllint disable-line rule:truthy # schedule: # - cron: '0 10-16 * * 1-5' workflow_dispatch: + inputs: + dry-run: + description: 'Dry run' + required: false + default: true + type: boolean jobs: AutoRelease: @@ -31,7 +38,7 @@ jobs: - name: Auto Release Prepare run: | cd "$GITHUB_WORKSPACE/tests/ci" - python3 auto_release.py + python3 auto_release.py --prepare echo "::group::Auto Release Info" python3 -m json.tool /tmp/autorelease_info.json echo "::endgroup::" @@ -40,14 +47,50 @@ jobs: cat /tmp/autorelease_info.json echo 'EOF' } >> "$GITHUB_ENV" + - name: Post Release Branch statuses + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 auto_release.py --post-status - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0].release_branch }} - if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0] }} - uses: ./.github/workflows/create_release.yml + if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0] && fromJson(env.AUTO_RELEASE_PARAMS).releases[0].ready }} + uses: ./.github/actions/release with: - type: patch ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0].commit_sha }} - dry-run: true - autorelease: true + type: patch + dry-run: ${{ inputs.dry-run }} + token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} + - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[1].release_branch }} + if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0] && fromJson(env.AUTO_RELEASE_PARAMS).releases[1].ready }} + uses: ./.github/actions/release + with: + ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[1].commit_sha }} + type: patch + dry-run: ${{ inputs.dry-run }} + token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} + - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2].release_branch }} + if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2] && fromJson(env.AUTO_RELEASE_PARAMS).releases[2].ready }} + uses: ./.github/actions/release + with: + ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2].commit_sha }} + type: patch + dry-run: ${{ inputs.dry-run }} + token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} + - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3].release_branch }} + if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3] && fromJson(env.AUTO_RELEASE_PARAMS).releases[3].ready }} + uses: ./.github/actions/release + with: + ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3].commit_sha }} + type: patch + dry-run: ${{ inputs.dry-run }} + token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} + - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4].release_branch }} + if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4] && fromJson(env.AUTO_RELEASE_PARAMS).releases[4].ready }} + uses: ./.github/actions/release + with: + ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4].commit_sha }} + type: patch + dry-run: ${{ inputs.dry-run }} + token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - name: Post Slack Message if: ${{ !cancelled() }} run: | diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 96cd46f583a..6246306e536 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -3,7 +3,7 @@ name: CreateRelease concurrency: group: release -'on': +on: workflow_dispatch: inputs: ref: @@ -22,10 +22,6 @@ concurrency: required: false default: true type: boolean - autorelease: - required: false - default: false - type: boolean jobs: CreateRelease: @@ -36,138 +32,16 @@ jobs: - name: DebugInfo if: ${{ ! inputs.autorelease }} uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6 - - name: Set envs - if: ${{ ! inputs.autorelease }} - # https://docs.github.com/en/actions/learn-github-actions/workflow-commands-for-github-actions#multiline-strings - run: | - cat >> "$GITHUB_ENV" << 'EOF' - ROBOT_CLICKHOUSE_SSH_KEY<> "$GITHUB_ENV" - echo "COMMIT_SHA=$commit_sha" >> "$GITHUB_ENV" - - name: Download All Release Artifacts - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/create_release.py --infile "$RELEASE_INFO_FILE" --download-packages ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Push Git Tag for the Release - run: | - python3 ./tests/ci/create_release.py --push-release-tag --infile "$RELEASE_INFO_FILE" ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Push New Release Branch - if: ${{ inputs.type == 'new' }} - run: | - python3 ./tests/ci/create_release.py --push-new-release-branch --infile "$RELEASE_INFO_FILE" ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Bump CH Version and Update Contributors' List - run: | - python3 ./tests/ci/create_release.py --create-bump-version-pr --infile "$RELEASE_INFO_FILE" ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Checkout master - run: | - git checkout master - - name: Bump Docker versions, Changelog, Security - if: ${{ inputs.type == 'patch' }} - run: | - [ "$(git branch --show-current)" != "master" ] && echo "not on the master" && exit 1 - echo "List versions" - ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv - echo "Update docker version" - ./utils/list-versions/update-docker-version.sh - echo "Generate ChangeLog" - export CI=1 - docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 -e CI=1 --network=host \ - --volume=".:/ClickHouse" clickhouse/style-test \ - /ClickHouse/tests/ci/changelog.py -v --debug-helpers \ - --gh-user-or-token="$GH_TOKEN" --jobs=5 \ - --output="/ClickHouse/docs/changelogs/${{ env.RELEASE_TAG }}.md" ${{ env.RELEASE_TAG }} - git add ./docs/changelogs/${{ env.RELEASE_TAG }}.md - echo "Generate Security" - python3 ./utils/security-generator/generate_security.py > SECURITY.md - git diff HEAD - - name: Create ChangeLog PR - if: ${{ inputs.type == 'patch' && ! inputs.dry-run }} - uses: peter-evans/create-pull-request@v6 + - name: Call Release Action + uses: ./.github/actions/release with: - author: "robot-clickhouse " - token: ${{ secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN }} - committer: "robot-clickhouse " - commit-message: Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} - branch: auto/${{ env.RELEASE_TAG }} - assignees: ${{ github.event.sender.login }} # assign the PR to the tag pusher - delete-branch: true - title: Update version_date.tsv and changelog after ${{ env.RELEASE_TAG }} - labels: do not test - body: | - Update version_date.tsv and changelogs after ${{ env.RELEASE_TAG }} - ### Changelog category (leave one): - - Not for changelog (changelog entry is not required) - - name: Reset changes if Dry-run - if: ${{ inputs.dry-run }} - run: | - git reset --hard HEAD - - name: Checkout back to GITHUB_REF - run: | - git checkout "$GITHUB_REF_NAME" - - name: Create GH Release - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/create_release.py --create-gh-release \ - --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }} - - - name: Export TGZ Packages - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/artifactory.py --export-tgz --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Test TGZ Packages - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/artifactory.py --test-tgz --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Export RPM Packages - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/artifactory.py --export-rpm --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Test RPM Packages - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/artifactory.py --test-rpm --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Export Debian Packages - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/artifactory.py --export-debian --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Test Debian Packages - if: ${{ inputs.type == 'patch' }} - run: | - python3 ./tests/ci/artifactory.py --test-debian --infile ${{ env.RELEASE_INFO_FILE }} ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Docker clickhouse/clickhouse-server building - if: ${{ inputs.type == 'patch' }} - run: | - cd "./tests/ci" - export CHECK_NAME="Docker server image" - python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} - - name: Docker clickhouse/clickhouse-keeper building - if: ${{ inputs.type == 'patch' }} - run: | - cd "./tests/ci" - export CHECK_NAME="Docker keeper image" - python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} - - name: Post Slack Message - if: ${{ !cancelled() }} - run: | - echo Slack Message + ref: ${{ inputs.ref }} + type: inputs.type + dry-run: ${{ inputs.dry-run }} + token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index 1a062d05a23..2009b122a18 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -4,7 +4,7 @@ from pathlib import Path from typing import Optional from shutil import copy2 from create_release import PackageDownloader, ReleaseInfo, ShellRunner -from ci_utils import WithIter +from ci_utils import WithIter, Shell class MountPointApp(metaclass=WithIter): @@ -141,12 +141,16 @@ class DebianArtifactory: ShellRunner.run("sync") def test_packages(self): - ShellRunner.run("docker pull ubuntu:latest") + Shell.run("docker pull ubuntu:latest") print(f"Test packages installation, version [{self.version}]") - cmd = f"docker run --rm ubuntu:latest bash -c \"apt update -y; apt install -y sudo gnupg ca-certificates; apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754; echo 'deb {self.repo_url} stable main' | tee /etc/apt/sources.list.d/clickhouse.list; apt update -y; apt-get install -y clickhouse-client={self.version}\"" + debian_command = f"echo 'deb {self.repo_url} stable main' | tee /etc/apt/sources.list.d/clickhouse.list; apt update -y; apt-get install -y clickhouse-common-static={self.version} clickhouse-client={self.version}" + cmd = f'docker run --rm ubuntu:latest bash -c "apt update -y; apt install -y sudo gnupg ca-certificates; apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754; {debian_command}"' print("Running test command:") print(f" {cmd}") - ShellRunner.run(cmd) + Shell.run(cmd, check=True) + release_info = ReleaseInfo.from_file() + release_info.debian_command = debian_command + release_info.dump() def _copy_if_not_exists(src: Path, dst: Path) -> Path: @@ -210,15 +214,19 @@ class RpmArtifactory: pub_key_path.write_text(ShellRunner.run(update_public_key)[1]) if codename == RepoCodenames.LTS: self.export_packages(RepoCodenames.STABLE) - ShellRunner.run("sync") + Shell.run("sync") def test_packages(self): - ShellRunner.run("docker pull fedora:latest") + Shell.run("docker pull fedora:latest") print(f"Test package installation, version [{self.version}]") - cmd = f'docker run --rm fedora:latest /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && dnf makecache && dnf -y install clickhouse-client-{self.version}-1"' + rpm_command = f"dnf config-manager --add-repo={self.repo_url} && dnf makecache && dnf -y install clickhouse-client-{self.version}-1" + cmd = f'docker run --rm fedora:latest /bin/bash -c "dnf -y install dnf-plugins-core && dnf config-manager --add-repo={self.repo_url} && {rpm_command}"' print("Running test command:") print(f" {cmd}") - ShellRunner.run(cmd) + Shell.run(cmd, check=True) + release_info = ReleaseInfo.from_file() + release_info.rpm_command = rpm_command + release_info.dump() class TgzArtifactory: @@ -280,12 +288,6 @@ def parse_args() -> argparse.Namespace: formatter_class=argparse.ArgumentDefaultsHelpFormatter, description="Adds release packages to the repository", ) - parser.add_argument( - "--infile", - type=str, - required=True, - help="input file with release info", - ) parser.add_argument( "--export-debian", action="store_true", @@ -328,7 +330,7 @@ if __name__ == "__main__": args = parse_args() assert args.dry_run - release_info = ReleaseInfo.from_file(args.infile) + release_info = ReleaseInfo.from_file() """ Use S3FS. RCLONE has some errors with r2 remote which I didn't figure out how to resolve: ERROR : IO error: NotImplemented: versionId not implemented diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index 88e91eb8aed..b8f781c4d73 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -1,20 +1,17 @@ import argparse import dataclasses import json -import logging import os +import sys from typing import List from get_robot_token import get_best_robot_token from github_helper import GitHub -from ssh import SSHKey from ci_utils import Shell from env_helper import GITHUB_REPOSITORY from report import SUCCESS - -LOGGER_NAME = __name__ -HELPER_LOGGERS = ["github_helper", LOGGER_NAME] -logger = logging.getLogger(LOGGER_NAME) +from ci_buddy import CIBuddy +from ci_config import CI def parse_args(): @@ -23,8 +20,17 @@ def parse_args(): "branches and do a release in case for green builds." ) parser.add_argument("--token", help="GitHub token, if not set, used from smm") - - return parser.parse_args() + parser.add_argument( + "--post-status", + action="store_true", + help="Post release branch statuses", + ) + parser.add_argument( + "--prepare", + action="store_true", + help="Prepare autorelease info", + ) + return parser.parse_args(), parser MAX_NUMBER_OF_COMMITS_TO_CONSIDER_FOR_RELEASE = 5 @@ -33,8 +39,16 @@ AUTORELEASE_INFO_FILE = "/tmp/autorelease_info.json" @dataclasses.dataclass class ReleaseParams: + ready: bool + ci_status: str + num_patches: int release_branch: str commit_sha: str + commits_to_branch_head: int + latest: bool + + def to_dict(self): + return dataclasses.asdict(self) @dataclasses.dataclass @@ -49,83 +63,128 @@ class AutoReleaseInfo: with open(AUTORELEASE_INFO_FILE, "w", encoding="utf-8") as f: print(json.dumps(dataclasses.asdict(self), indent=2), file=f) + @staticmethod + def from_file() -> "AutoReleaseInfo": + with open(AUTORELEASE_INFO_FILE, "r", encoding="utf-8") as json_file: + res = json.load(json_file) + releases = [ReleaseParams(**release) for release in res["releases"]] + return AutoReleaseInfo(releases=releases) -def main(): - args = parse_args() - token = args.token or get_best_robot_token() +def _prepare(token): assert len(token) > 10 os.environ["GH_TOKEN"] = token - (Shell.run("gh auth status", check=True)) + Shell.run("gh auth status", check=True) + gh = GitHub(token) prs = gh.get_release_pulls(GITHUB_REPOSITORY) + prs.sort(key=lambda x: x.head.ref) branch_names = [pr.head.ref for pr in prs] - print(f"Found release branches [{branch_names}]") - repo = gh.get_repo(GITHUB_REPOSITORY) + repo = gh.get_repo(GITHUB_REPOSITORY) autoRelease_info = AutoReleaseInfo(releases=[]) + for pr in prs: - print(f"Checking PR [{pr.head.ref}]") + print(f"\nChecking PR [{pr.head.ref}]") refs = list(repo.get_git_matching_refs(f"tags/v{pr.head.ref}")) - refs.sort(key=lambda ref: ref.ref) + assert refs + refs.sort(key=lambda ref: ref.ref) latest_release_tag_ref = refs[-1] latest_release_tag = repo.get_git_tag(latest_release_tag_ref.object.sha) - commit_num = int( - Shell.run( - f"git rev-list --count {latest_release_tag.tag}..origin/{pr.head.ref}", - check=True, - ) - ) - print( - f"Previous release is [{latest_release_tag}] was [{commit_num}] commits before, date [{latest_release_tag.tagger.date}]" - ) - commit_reverse_index = 0 - commit_found = False - commit_checked = False - commit_sha = "" - while ( - commit_reverse_index < commit_num - 1 - and commit_reverse_index < MAX_NUMBER_OF_COMMITS_TO_CONSIDER_FOR_RELEASE - ): - commit_checked = True - commit_sha = Shell.run( - f"git rev-list --max-count=1 --skip={commit_reverse_index} origin/{pr.head.ref}", - check=True, - ) - print( - f"Check if commit [{commit_sha}] [{pr.head.ref}~{commit_reverse_index}] is ready for release" - ) - commit_reverse_index += 1 - cmd = f"gh api -H 'Accept: application/vnd.github.v3+json' /repos/{GITHUB_REPOSITORY}/commits/{commit_sha}/status" - ci_status_json = Shell.run(cmd, check=True) - ci_status = json.loads(ci_status_json)["state"] - if ci_status == SUCCESS: - commit_found = True - break - if commit_found: + commits = Shell.run( + f"git rev-list --first-parent {latest_release_tag.tag}..origin/{pr.head.ref}", + check=True, + ).split("\n") + commit_num = len(commits) + print( + f"Previous release [{latest_release_tag.tag}] was [{commit_num}] commits ago, date [{latest_release_tag.tagger.date}]" + ) + + commits_to_check = commits[:-1] # Exclude the version bump commit + commit_sha = "" + commit_ci_status = "" + commits_to_branch_head = 0 + + for idx, commit in enumerate( + commits_to_check[:MAX_NUMBER_OF_COMMITS_TO_CONSIDER_FOR_RELEASE] + ): + print( + f"Check commit [{commit}] [{pr.head.ref}~{idx+1}] as release candidate" + ) + commit_num -= 1 + + is_completed = CI.GHActions.check_wf_completed( + token=token, commit_sha=commit + ) + if not is_completed: + print(f"CI is in progress for [{commit}] - check previous commit") + commits_to_branch_head += 1 + continue + + commit_ci_status = CI.GHActions.get_commit_status_by_name( + token=token, + commit_sha=commit, + status_name=(CI.JobNames.BUILD_CHECK, "ClickHouse build check"), + ) + commit_sha = commit + if commit_ci_status == SUCCESS: + break + else: + print(f"CI status [{commit_ci_status}] - skip") + commits_to_branch_head += 1 + + ready = commit_ci_status == SUCCESS and commit_sha + if ready: print( f"Add release ready info for commit [{commit_sha}] and release branch [{pr.head.ref}]" ) - autoRelease_info.add_release( - ReleaseParams(release_branch=pr.head.ref, commit_sha=commit_sha) - ) else: - print(f"WARNING: No good commits found for release branch [{pr.head.ref}]") - if commit_checked: - print( - f"ERROR: CI is failed. check CI status for branch [{pr.head.ref}]" - ) + print(f"WARNING: No ready commits found for release branch [{pr.head.ref}]") + + autoRelease_info.add_release( + ReleaseParams( + release_branch=pr.head.ref, + commit_sha=commit_sha, + ready=ready, + ci_status=commit_ci_status, + num_patches=commit_num, + commits_to_branch_head=commits_to_branch_head, + latest=False, + ) + ) + + if autoRelease_info.releases: + autoRelease_info.releases[-1].latest = True autoRelease_info.dump() -if __name__ == "__main__": - if os.getenv("ROBOT_CLICKHOUSE_SSH_KEY", ""): - with SSHKey("ROBOT_CLICKHOUSE_SSH_KEY"): - main() +def main(): + args, parser = parse_args() + + if args.post_status: + info = AutoReleaseInfo.from_file() + for release_info in info.releases: + if release_info.ready: + CIBuddy(dry_run=False).post_info( + title=f"Auto Release Status for {release_info.release_branch}", + body=release_info.to_dict(), + ) + else: + CIBuddy(dry_run=False).post_warning( + title=f"Auto Release Status for {release_info.release_branch}", + body=release_info.to_dict(), + ) + elif args.prepare: + _prepare(token=args.token or get_best_robot_token()) else: - main() + parser.print_help() + sys.exit(2) + + +if __name__ == "__main__": + main() diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 110a7b2a49c..8ad358cb874 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1295,7 +1295,7 @@ def main() -> int: error_description = f"Out Of Memory, exit_code {job_report.exit_code}" else: error_description = f"Unknown, exit_code {job_report.exit_code}" - CIBuddy().post_error( + CIBuddy().post_job_error( error_description + f" after {int(job_report.duration)}s", job_name=_get_ext_check_name(args.job_name), ) diff --git a/tests/ci/ci_buddy.py b/tests/ci/ci_buddy.py index c650b876610..727a3d88359 100644 --- a/tests/ci/ci_buddy.py +++ b/tests/ci/ci_buddy.py @@ -1,5 +1,6 @@ import json import os +from typing import Union, Dict import boto3 import requests @@ -60,7 +61,55 @@ class CIBuddy: except Exception as e: print(f"ERROR: Failed to post message, ex {e}") - def post_error(self, error_description, job_name="", with_instance_info=True): + def _post_formatted( + self, title, body: Union[Dict, str], with_wf_link: bool + ) -> None: + message = title + if isinstance(body, dict): + for name, value in body.items(): + if "commit_sha" in name: + value = ( + f"" + ) + message += f" *{name}*: {value}\n" + else: + message += body + "\n" + run_id = os.getenv("GITHUB_RUN_ID", "") + if with_wf_link and run_id: + message += f" *workflow*: \n" + self.post(message) + + def post_info( + self, title, body: Union[Dict, str], with_wf_link: bool = True + ) -> None: + title_extended = f":white_circle: *{title}*\n\n" + self._post_formatted(title_extended, body, with_wf_link) + + def post_done( + self, title, body: Union[Dict, str], with_wf_link: bool = True + ) -> None: + title_extended = f":white_check_mark: *{title}*\n\n" + self._post_formatted(title_extended, body, with_wf_link) + + def post_warning( + self, title, body: Union[Dict, str], with_wf_link: bool = True + ) -> None: + title_extended = f":warning: *{title}*\n\n" + self._post_formatted(title_extended, body, with_wf_link) + + def post_critical( + self, title, body: Union[Dict, str], with_wf_link: bool = True + ) -> None: + title_extended = f":black_circle: *{title}*\n\n" + self._post_formatted(title_extended, body, with_wf_link) + + def post_job_error( + self, + error_description, + job_name="", + with_instance_info=True, + with_wf_link: bool = True, + ): instance_id, instance_type = "unknown", "unknown" if with_instance_info: instance_id = Shell.run("ec2metadata --instance-id") or instance_id @@ -82,10 +131,13 @@ class CIBuddy: message += line_pr_ else: message += line_br_ + run_id = os.getenv("GITHUB_RUN_ID", "") + if with_wf_link and run_id: + message += f" *workflow*: \n" self.post(message) if __name__ == "__main__": # test buddy = CIBuddy(dry_run=True) - buddy.post_error("TEst") + buddy.post_job_error("TEst") diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 0a4ab3a823b..a44b15f34c1 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -32,6 +32,9 @@ class CI: from ci_definitions import MQ_JOBS as MQ_JOBS from ci_definitions import WorkflowStages as WorkflowStages from ci_definitions import Runners as Runners + from ci_utils import Envs as Envs + from ci_utils import Utils as Utils + from ci_utils import GHActions as GHActions from ci_definitions import Labels as Labels from ci_definitions import TRUSTED_CONTRIBUTORS as TRUSTED_CONTRIBUTORS from ci_utils import CATEGORY_TO_LABEL as CATEGORY_TO_LABEL diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 0653374356f..4536d1f2b54 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -1,9 +1,16 @@ import os import re import subprocess +import time from contextlib import contextmanager from pathlib import Path -from typing import Any, Iterator, List, Union, Optional, Tuple +from typing import Any, Iterator, List, Union, Optional, Sequence + +import requests + + +class Envs: + GITHUB_REPOSITORY = os.getenv("GITHUB_REPOSITORY", "ClickHouse/ClickHouse") LABEL_CATEGORIES = { @@ -80,6 +87,62 @@ class GHActions: print(line) print("::endgroup::") + @staticmethod + def get_commit_status_by_name( + token: str, commit_sha: str, status_name: Union[str, Sequence] + ) -> Optional[str]: + assert len(token) == 40 + assert len(commit_sha) == 40 + assert is_hex(commit_sha) + assert not is_hex(token) + url = f"https://api.github.com/repos/{Envs.GITHUB_REPOSITORY}/commits/{commit_sha}/statuses?per_page={200}" + headers = { + "Authorization": f"token {token}", + "Accept": "application/vnd.github.v3+json", + } + response = requests.get(url, headers=headers, timeout=5) + + if isinstance(status_name, str): + status_name = (status_name,) + if response.status_code == 200: + assert "next" not in response.links, "Response truncated" + statuses = response.json() + for status in statuses: + if status["context"] in status_name: + return status["state"] + return None + + @staticmethod + def check_wf_completed(token: str, commit_sha: str) -> bool: + headers = { + "Authorization": f"token {token}", + "Accept": "application/vnd.github.v3+json", + } + url = f"https://api.github.com/repos/{Envs.GITHUB_REPOSITORY}/commits/{commit_sha}/check-runs?per_page={100}" + + for i in range(3): + try: + response = requests.get(url, headers=headers, timeout=5) + response.raise_for_status() + # assert "next" not in response.links, "Response truncated" + + data = response.json() + assert data["check_runs"], "?" + + for check in data["check_runs"]: + if check["status"] != "completed": + print( + f" Check workflow status: Check not completed [{check['name']}]" + ) + return False + else: + return True + except Exception as e: + print(f"ERROR: exception {e}") + time.sleep(1) + + return False + class Shell: @classmethod @@ -108,15 +171,18 @@ class Shell: ) if result.returncode == 0: res = result.stdout - elif check: - print(f"ERROR: stdout {result.stdout}, stderr {result.stderr}") - assert result.returncode == 0 + else: + print( + f"ERROR: stdout {result.stdout.strip()}, stderr {result.stderr.strip()}" + ) + if check: + assert result.returncode == 0 return res.strip() @classmethod def check(cls, command): result = subprocess.run( - command + " 2>&1", + command, shell=True, stdout=subprocess.PIPE, stderr=subprocess.PIPE, diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index 277134c3991..414ec8afd3e 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -14,6 +14,7 @@ from ssh import SSHAgent from env_helper import GITHUB_REPOSITORY, S3_BUILDS_BUCKET from s3_helper import S3Helper from ci_utils import Shell +from ci_buddy import CIBuddy from version_helper import ( FILE_WITH_VERSION_PATH, GENERATED_CONTRIBUTORS, @@ -27,6 +28,7 @@ from ci_config import CI CMAKE_PATH = get_abs_path(FILE_WITH_VERSION_PATH) CONTRIBUTORS_PATH = get_abs_path(GENERATED_CONTRIBUTORS) +RELEASE_INFO_FILE = "/tmp/release_info.json" class ShellRunner: @@ -67,17 +69,25 @@ class ReleaseInfo: codename: str previous_release_tag: str previous_release_sha: str + changelog_pr: str = "" + version_bump_pr: str = "" + release_url: str = "" + debian_command: str = "" + rpm_command: str = "" @staticmethod - def from_file(file_path: str) -> "ReleaseInfo": - with open(file_path, "r", encoding="utf-8") as json_file: + def from_file() -> "ReleaseInfo": + with open(RELEASE_INFO_FILE, "r", encoding="utf-8") as json_file: res = json.load(json_file) return ReleaseInfo(**res) + def dump(self): + print(f"Dump release info into [{RELEASE_INFO_FILE}]") + with open(RELEASE_INFO_FILE, "w", encoding="utf-8") as f: + print(json.dumps(dataclasses.asdict(self), indent=2), file=f) + @staticmethod - def prepare(commit_ref: str, release_type: str, outfile: str) -> None: - Path(outfile).parent.mkdir(parents=True, exist_ok=True) - Path(outfile).unlink(missing_ok=True) + def prepare(commit_ref: str, release_type: str) -> None: version = None release_branch = None release_tag = None @@ -91,7 +101,7 @@ class ReleaseInfo: f"git merge-base --is-ancestor origin/{commit_ref} origin/master" ) with checkout(commit_ref): - _, commit_sha = ShellRunner.run(f"git rev-parse {commit_ref}") + commit_sha = Shell.run(f"git rev-parse {commit_ref}", check=True) # Git() must be inside "with checkout" contextmanager git = Git() version = get_version_from_repo(git=git) @@ -112,7 +122,7 @@ class ReleaseInfo: assert previous_release_sha if release_type == "patch": with checkout(commit_ref): - _, commit_sha = ShellRunner.run(f"git rev-parse {commit_ref}") + commit_sha = Shell.run(f"git rev-parse {commit_ref}", check=True) # Git() must be inside "with checkout" contextmanager git = Git() version = get_version_from_repo(git=git) @@ -171,8 +181,7 @@ class ReleaseInfo: previous_release_tag=previous_release_tag, previous_release_sha=previous_release_sha, ) - with open(outfile, "w", encoding="utf-8") as f: - print(json.dumps(dataclasses.asdict(res), indent=2), file=f) + res.dump() def push_release_tag(self, dry_run: bool) -> None: if dry_run: @@ -276,21 +285,38 @@ class ReleaseInfo: f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'" ) + def update_release_info(self, dry_run: bool) -> None: + branch = f"auto/{release_info.release_tag}" + if not dry_run: + get_url_cmd = f"gh pr list --repo {GITHUB_REPOSITORY} --head {branch} --json url --jq '.[0].url'" + url = Shell.run(get_url_cmd) + if url: + print(f"Update release info with Changelog PR link [{url}]") + else: + print(f"WARNING: Changelog PR not found, branch [{branch}]") + else: + url = "dry-run" + + self.changelog_pr = url + self.dump() + def create_gh_release(self, packages_files: List[str], dry_run: bool) -> None: repo = os.getenv("GITHUB_REPOSITORY") assert repo - cmds = [] - cmds.append( + cmds = [ f"gh release create --repo {repo} --title 'Release {self.release_tag}' {self.release_tag}" - ) + ] for file in packages_files: cmds.append(f"gh release upload {self.release_tag} {file}") if not dry_run: for cmd in cmds: - ShellRunner.run(cmd) + Shell.run(cmd, check=True) + self.release_url = f"https://github.com/{GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" else: print("Dry-run, would run commands:") print("\n * ".join(cmds)) + self.release_url = f"dry-run" + self.dump() class RepoTypes: @@ -508,6 +534,11 @@ def parse_args() -> argparse.Namespace: action="store_true", help="Create GH Release object and attach all packages", ) + parser.add_argument( + "--post-status", + action="store_true", + help="Post release status into Slack", + ) parser.add_argument( "--ref", type=str, @@ -525,18 +556,6 @@ def parse_args() -> argparse.Namespace: action="store_true", help="do not make any actual changes in the repo, just show what will be done", ) - parser.add_argument( - "--outfile", - default="", - type=str, - help="output file to write json result to, if not set - stdout", - ) - parser.add_argument( - "--infile", - default="", - type=str, - help="input file with release info", - ) return parser.parse_args() @@ -547,7 +566,7 @@ def checkout(ref: str) -> Iterator[None]: rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}" assert orig_ref if ref not in (orig_ref,): - ShellRunner.run(f"{GIT_PREFIX} checkout {ref}") + Shell.run(f"{GIT_PREFIX} checkout {ref}") try: yield except (Exception, KeyboardInterrupt) as e: @@ -587,27 +606,21 @@ if __name__ == "__main__": if args.prepare_release_info: assert ( - args.ref and args.release_type and args.outfile - ), "--ref, --release-type and --outfile must be provided with --prepare-release-info" - ReleaseInfo.prepare( - commit_ref=args.ref, release_type=args.release_type, outfile=args.outfile - ) + args.ref and args.release_type + ), "--ref and --release-type must be provided with --prepare-release-info" + ReleaseInfo.prepare(commit_ref=args.ref, release_type=args.release_type) if args.push_release_tag: - assert args.infile, "--infile must be provided" - release_info = ReleaseInfo.from_file(args.infile) + release_info = ReleaseInfo.from_file() release_info.push_release_tag(dry_run=args.dry_run) if args.push_new_release_branch: - assert args.infile, "--infile must be provided" - release_info = ReleaseInfo.from_file(args.infile) + release_info = ReleaseInfo.from_file() release_info.push_new_release_branch(dry_run=args.dry_run) if args.create_bump_version_pr: # TODO: store link to PR in release info - assert args.infile, "--infile must be provided" - release_info = ReleaseInfo.from_file(args.infile) + release_info = ReleaseInfo.from_file() release_info.update_version_and_contributors_list(dry_run=args.dry_run) if args.download_packages: - assert args.infile, "--infile must be provided" - release_info = ReleaseInfo.from_file(args.infile) + release_info = ReleaseInfo.from_file() p = PackageDownloader( release=release_info.release_branch, commit_sha=release_info.commit_sha, @@ -615,14 +628,23 @@ if __name__ == "__main__": ) p.run() if args.create_gh_release: - assert args.infile, "--infile must be provided" - release_info = ReleaseInfo.from_file(args.infile) + release_info = ReleaseInfo.from_file() p = PackageDownloader( release=release_info.release_branch, commit_sha=release_info.commit_sha, version=release_info.version, ) - release_info.create_gh_release(p.get_all_packages_files(), args.dry_run) + if args.post_status: + release_info = ReleaseInfo.from_file() + release_info.update_release_info(dry_run=args.dry_run) + if release_info.debian_command: + CIBuddy(dry_run=args.dry_run).post_done( + f"New release issued", dataclasses.asdict(release_info) + ) + else: + CIBuddy(dry_run=args.dry_run).post_critical( + f"Failed to issue new release", dataclasses.asdict(release_info) + ) # tear down ssh if _ssh_agent and _key_pub: diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 9f3b5a586cc..59806a2a8fa 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -296,13 +296,16 @@ class PRInfo: else: if "schedule" in github_event: self.event_type = EventType.SCHEDULE - else: + elif "inputs" in github_event: # assume this is a dispatch self.event_type = EventType.DISPATCH - logging.warning( - "event.json does not match pull_request or push:\n%s", - json.dumps(github_event, sort_keys=True, indent=4), - ) + print("PR Info:") + print(self) + else: + logging.warning( + "event.json does not match pull_request or push:\n%s", + json.dumps(github_event, sort_keys=True, indent=4), + ) self.sha = os.getenv( "GITHUB_SHA", "0000000000000000000000000000000000000000" ) From 952ab302ce1fa5d9b739bbdf6acdf2fcdd208a04 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 19 Jul 2024 09:00:11 +0200 Subject: [PATCH 10/26] run test auto release by schedule --- .github/workflows/auto_release.yml | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/.github/workflows/auto_release.yml b/.github/workflows/auto_release.yml index e0a94d3bbb1..7588e7998bf 100644 --- a/.github/workflows/auto_release.yml +++ b/.github/workflows/auto_release.yml @@ -7,8 +7,10 @@ env: concurrency: group: release on: # yamllint disable-line rule:truthy - # schedule: - # - cron: '0 10-16 * * 1-5' + # Workflow uses a test bucket for packages and dry run mode (no real releases) + schedule: + - cron: '0 9 * * *' + - cron: '0 15 * * *' workflow_dispatch: inputs: dry-run: @@ -19,6 +21,8 @@ on: # yamllint disable-line rule:truthy jobs: AutoRelease: + env: + DRY_RUN: ${{ github.event_name == 'workflow_dispatch' && github.event.inputs.dry_run != '' && github.event.inputs.dry_run || true }} runs-on: [self-hosted, release-maker] steps: - name: DebugInfo @@ -57,7 +61,7 @@ jobs: with: ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0].commit_sha }} type: patch - dry-run: ${{ inputs.dry-run }} + dry-run: ${{ env.DRY_RUN }} token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[1].release_branch }} if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[0] && fromJson(env.AUTO_RELEASE_PARAMS).releases[1].ready }} @@ -65,7 +69,7 @@ jobs: with: ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[1].commit_sha }} type: patch - dry-run: ${{ inputs.dry-run }} + dry-run: ${{ env.DRY_RUN }} token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2].release_branch }} if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2] && fromJson(env.AUTO_RELEASE_PARAMS).releases[2].ready }} @@ -73,7 +77,7 @@ jobs: with: ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[2].commit_sha }} type: patch - dry-run: ${{ inputs.dry-run }} + dry-run: ${{ env.DRY_RUN }} token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3].release_branch }} if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3] && fromJson(env.AUTO_RELEASE_PARAMS).releases[3].ready }} @@ -81,7 +85,7 @@ jobs: with: ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[3].commit_sha }} type: patch - dry-run: ${{ inputs.dry-run }} + dry-run: ${{ env.DRY_RUN }} token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - name: Release ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4].release_branch }} if: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4] && fromJson(env.AUTO_RELEASE_PARAMS).releases[4].ready }} @@ -89,7 +93,7 @@ jobs: with: ref: ${{ fromJson(env.AUTO_RELEASE_PARAMS).releases[4].commit_sha }} type: patch - dry-run: ${{ inputs.dry-run }} + dry-run: ${{ env.DRY_RUN }} token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} - name: Post Slack Message if: ${{ !cancelled() }} From 5d09f205e5bb5ab8aa860b14334eb5656b2c2a1b Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Fri, 19 Jul 2024 09:35:43 +0000 Subject: [PATCH 11/26] style fixes --- .github/workflows/auto_release.yml | 8 ++++++-- .github/workflows/create_release.yml | 5 +---- tests/ci/auto_release.py | 11 ++++++----- tests/ci/ci_buddy.py | 18 +++++++++--------- tests/ci/ci_utils.py | 11 +++++------ 5 files changed, 27 insertions(+), 26 deletions(-) diff --git a/.github/workflows/auto_release.yml b/.github/workflows/auto_release.yml index 7588e7998bf..f2cbf771190 100644 --- a/.github/workflows/auto_release.yml +++ b/.github/workflows/auto_release.yml @@ -21,8 +21,6 @@ on: # yamllint disable-line rule:truthy jobs: AutoRelease: - env: - DRY_RUN: ${{ github.event_name == 'workflow_dispatch' && github.event.inputs.dry_run != '' && github.event.inputs.dry_run || true }} runs-on: [self-hosted, release-maker] steps: - name: DebugInfo @@ -34,6 +32,12 @@ jobs: ${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} RCSK EOF + - name: Set DRY_RUN for schedule + if: ${{ github.event_name == 'schedule' }} + run: echo "DRY_RUN=true" >> "$GITHUB_ENV" + - name: Set DRY_RUN for dispatch + if: ${{ github.event_name == 'workflow_dispatch' }} + run: echo "DRY_RUN=${{ github.event.inputs.dry-run }}" >> "$GITHUB_ENV" - name: Check out repository code uses: ClickHouse/checkout@v1 with: diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 6246306e536..6d914d1567e 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -2,8 +2,7 @@ name: CreateRelease concurrency: group: release - -on: +'on': workflow_dispatch: inputs: ref: @@ -30,10 +29,8 @@ jobs: runs-on: [self-hosted, release-maker] steps: - name: DebugInfo - if: ${{ ! inputs.autorelease }} uses: hmarr/debug-action@f7318c783045ac39ed9bb497e22ce835fdafbfe6 - name: Check out repository code - if: ${{ ! inputs.autorelease }} uses: ClickHouse/checkout@v1 with: token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index b8f781c4d73..5d98d8810a4 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -55,7 +55,7 @@ class ReleaseParams: class AutoReleaseInfo: releases: List[ReleaseParams] - def add_release(self, release_params: ReleaseParams): + def add_release(self, release_params: ReleaseParams) -> None: self.releases.append(release_params) def dump(self): @@ -133,15 +133,16 @@ def _prepare(token): commit_sha = commit if commit_ci_status == SUCCESS: break - else: - print(f"CI status [{commit_ci_status}] - skip") + + print(f"CI status [{commit_ci_status}] - skip") commits_to_branch_head += 1 - ready = commit_ci_status == SUCCESS and commit_sha - if ready: + ready = False + if commit_ci_status == SUCCESS and commit_sha: print( f"Add release ready info for commit [{commit_sha}] and release branch [{pr.head.ref}]" ) + ready = True else: print(f"WARNING: No ready commits found for release branch [{pr.head.ref}]") diff --git a/tests/ci/ci_buddy.py b/tests/ci/ci_buddy.py index 727a3d88359..ff8fdba2b6c 100644 --- a/tests/ci/ci_buddy.py +++ b/tests/ci/ci_buddy.py @@ -62,7 +62,7 @@ class CIBuddy: print(f"ERROR: Failed to post message, ex {e}") def _post_formatted( - self, title, body: Union[Dict, str], with_wf_link: bool + self, title: str, body: Union[Dict, str], with_wf_link: bool ) -> None: message = title if isinstance(body, dict): @@ -80,36 +80,36 @@ class CIBuddy: self.post(message) def post_info( - self, title, body: Union[Dict, str], with_wf_link: bool = True + self, title: str, body: Union[Dict, str], with_wf_link: bool = True ) -> None: title_extended = f":white_circle: *{title}*\n\n" self._post_formatted(title_extended, body, with_wf_link) def post_done( - self, title, body: Union[Dict, str], with_wf_link: bool = True + self, title: str, body: Union[Dict, str], with_wf_link: bool = True ) -> None: title_extended = f":white_check_mark: *{title}*\n\n" self._post_formatted(title_extended, body, with_wf_link) def post_warning( - self, title, body: Union[Dict, str], with_wf_link: bool = True + self, title: str, body: Union[Dict, str], with_wf_link: bool = True ) -> None: title_extended = f":warning: *{title}*\n\n" self._post_formatted(title_extended, body, with_wf_link) def post_critical( - self, title, body: Union[Dict, str], with_wf_link: bool = True + self, title: str, body: Union[Dict, str], with_wf_link: bool = True ) -> None: title_extended = f":black_circle: *{title}*\n\n" self._post_formatted(title_extended, body, with_wf_link) def post_job_error( self, - error_description, - job_name="", - with_instance_info=True, + error_description: str, + job_name: str = "", + with_instance_info: bool = True, with_wf_link: bool = True, - ): + ) -> None: instance_id, instance_type = "unknown", "unknown" if with_instance_info: instance_id = Shell.run("ec2metadata --instance-id") or instance_id diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index 4536d1f2b54..eb25a53d492 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -90,7 +90,7 @@ class GHActions: @staticmethod def get_commit_status_by_name( token: str, commit_sha: str, status_name: Union[str, Sequence] - ) -> Optional[str]: + ) -> str: assert len(token) == 40 assert len(commit_sha) == 40 assert is_hex(commit_sha) @@ -109,8 +109,8 @@ class GHActions: statuses = response.json() for status in statuses: if status["context"] in status_name: - return status["state"] - return None + return status["state"] # type: ignore + return "" @staticmethod def check_wf_completed(token: str, commit_sha: str) -> bool: @@ -135,10 +135,9 @@ class GHActions: f" Check workflow status: Check not completed [{check['name']}]" ) return False - else: - return True + return True except Exception as e: - print(f"ERROR: exception {e}") + print(f"ERROR: exception after attempt [{i}]: {e}") time.sleep(1) return False From 3b842885779e123af1fd8aeaca4e7c131c5a33a9 Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 19 Jul 2024 12:05:19 +0200 Subject: [PATCH 12/26] fix create release --- .github/workflows/create_release.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/create_release.yml b/.github/workflows/create_release.yml index 6d914d1567e..55644bdd503 100644 --- a/.github/workflows/create_release.yml +++ b/.github/workflows/create_release.yml @@ -39,6 +39,6 @@ jobs: uses: ./.github/actions/release with: ref: ${{ inputs.ref }} - type: inputs.type + type: ${{ inputs.type }} dry-run: ${{ inputs.dry-run }} token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} From 3767f723489507dad7fe275c245253e7885aab8e Mon Sep 17 00:00:00 2001 From: Max K Date: Fri, 19 Jul 2024 20:43:14 +0200 Subject: [PATCH 13/26] more info for buddy --- .github/actions/release/action.yml | 20 +- .github/workflows/auto_release.yml | 3 +- tests/ci/artifactory.py | 86 ++++--- tests/ci/auto_release.py | 27 +++ tests/ci/ci_buddy.py | 4 +- tests/ci/ci_utils.py | 23 +- tests/ci/create_release.py | 348 ++++++++++++++++++----------- 7 files changed, 338 insertions(+), 173 deletions(-) diff --git a/.github/actions/release/action.yml b/.github/actions/release/action.yml index fd4436f1f16..99ec02662f6 100644 --- a/.github/actions/release/action.yml +++ b/.github/actions/release/action.yml @@ -58,14 +58,11 @@ runs: shell: bash run: | python3 ./tests/ci/create_release.py --create-bump-version-pr ${{ inputs.dry-run && '--dry-run' || '' }} - - name: Checkout master - shell: bash - run: | - git checkout master - name: Bump Docker versions, Changelog, Security if: ${{ inputs.type == 'patch' }} shell: bash run: | + python3 ./tests/ci/create_release.py --set-progress-started --progress "update ChangeLog" [ "$(git branch --show-current)" != "master" ] && echo "not on the master" && exit 1 echo "List versions" ./utils/list-versions/list-versions.sh > ./utils/list-versions/version_date.tsv @@ -108,12 +105,13 @@ runs: shell: bash run: | git checkout "$GITHUB_REF_NAME" + # set current progress to OK + python3 ./tests/ci/create_release.py --set-progress-completed - name: Create GH Release shell: bash if: ${{ inputs.type == 'patch' }} run: | - python3 ./tests/ci/create_release.py --create-gh-release \ - ${{ inputs.dry-run && '--dry-run' || '' }} + python3 ./tests/ci/create_release.py --create-gh-release ${{ inputs.dry-run && '--dry-run' || '' }} - name: Export TGZ Packages if: ${{ inputs.type == 'patch' }} shell: bash @@ -148,16 +146,26 @@ runs: if: ${{ inputs.type == 'patch' }} shell: bash run: | + python3 ./tests/ci/create_release.py --set-progress-started --progress "docker server release" cd "./tests/ci" export CHECK_NAME="Docker server image" python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + python3 ./tests/ci/create_release.py --set-progress-completed - name: Docker clickhouse/clickhouse-keeper building if: ${{ inputs.type == 'patch' }} shell: bash run: | + python3 ./tests/ci/create_release.py --set-progress-started --progress "docker keeper release" cd "./tests/ci" export CHECK_NAME="Docker keeper image" python3 docker_server.py --release-type auto --version ${{ env.RELEASE_TAG }} --check-name "$CHECK_NAME" --sha ${{ env.COMMIT_SHA }} ${{ ! inputs.dry-run && '--push' || '' }} + python3 ./tests/ci/create_release.py --set-progress-completed + - name: Set Release progress completed + shell: bash + run: | + # If we here - set completed status, to post proper Slack OK or FAIL message in the next step + python3 ./tests/ci/create_release.py --set-progress-started --progress "completed" + python3 ./tests/ci/create_release.py --set-progress-completed - name: Post Slack Message if: ${{ !cancelled() }} shell: bash diff --git a/.github/workflows/auto_release.yml b/.github/workflows/auto_release.yml index f2cbf771190..457ffacc7a8 100644 --- a/.github/workflows/auto_release.yml +++ b/.github/workflows/auto_release.yml @@ -102,7 +102,8 @@ jobs: - name: Post Slack Message if: ${{ !cancelled() }} run: | - echo Slack Message + cd "$GITHUB_WORKSPACE/tests/ci" + python3 auto_release.py --post-auto-release-complete --wf-status ${{ job.status }} - name: Clean up run: | docker ps --quiet | xargs --no-run-if-empty docker kill ||: diff --git a/tests/ci/artifactory.py b/tests/ci/artifactory.py index 2009b122a18..98a0345c6bd 100644 --- a/tests/ci/artifactory.py +++ b/tests/ci/artifactory.py @@ -3,7 +3,12 @@ import time from pathlib import Path from typing import Optional from shutil import copy2 -from create_release import PackageDownloader, ReleaseInfo, ShellRunner +from create_release import ( + PackageDownloader, + ReleaseInfo, + ReleaseContextManager, + ReleaseProgress, +) from ci_utils import WithIter, Shell @@ -76,19 +81,20 @@ class R2MountPoint: ) _TEST_MOUNT_CMD = f"mount | grep -q {self.MOUNT_POINT}" - ShellRunner.run(_CLEAN_LOG_FILE_CMD) - ShellRunner.run(_UNMOUNT_CMD) - ShellRunner.run(_MKDIR_CMD) - ShellRunner.run(_MKDIR_FOR_CACHE) - ShellRunner.run(self.mount_cmd, async_=self.async_mount) + Shell.run(_CLEAN_LOG_FILE_CMD) + Shell.run(_UNMOUNT_CMD) + Shell.run(_MKDIR_CMD) + Shell.run(_MKDIR_FOR_CACHE) + # didn't manage to use simple run() and not block or fail + Shell.run_as_daemon(self.mount_cmd) if self.async_mount: time.sleep(3) - ShellRunner.run(_TEST_MOUNT_CMD) + Shell.run(_TEST_MOUNT_CMD, check=True) @classmethod def teardown(cls): print(f"Unmount [{cls.MOUNT_POINT}]") - ShellRunner.run(f"umount {cls.MOUNT_POINT}") + Shell.run(f"umount {cls.MOUNT_POINT}") class RepoCodenames(metaclass=WithIter): @@ -124,8 +130,8 @@ class DebianArtifactory: cmd = f"{REPREPRO_CMD_PREFIX} includedeb {self.codename} {' '.join(paths)}" print("Running export command:") print(f" {cmd}") - ShellRunner.run(cmd) - ShellRunner.run("sync") + Shell.run(cmd, check=True) + Shell.run("sync") if self.codename == RepoCodenames.LTS: packages_with_version = [ @@ -137,8 +143,8 @@ class DebianArtifactory: cmd = f"{REPREPRO_CMD_PREFIX} copy {RepoCodenames.STABLE} {RepoCodenames.LTS} {' '.join(packages_with_version)}" print("Running copy command:") print(f" {cmd}") - ShellRunner.run(cmd) - ShellRunner.run("sync") + Shell.run(cmd, check=True) + Shell.run("sync") def test_packages(self): Shell.run("docker pull ubuntu:latest") @@ -206,12 +212,12 @@ class RpmArtifactory: for command in commands: print("Running command:") print(f" {command}") - ShellRunner.run(command) + Shell.run(command, check=True) update_public_key = f"gpg --armor --export {self._SIGN_KEY}" pub_key_path = dest_dir / "repodata" / "repomd.xml.key" print("Updating repomd.xml.key") - pub_key_path.write_text(ShellRunner.run(update_public_key)[1]) + pub_key_path.write_text(Shell.run(update_public_key, check=True)) if codename == RepoCodenames.LTS: self.export_packages(RepoCodenames.STABLE) Shell.run("sync") @@ -264,23 +270,29 @@ class TgzArtifactory: if codename == RepoCodenames.LTS: self.export_packages(RepoCodenames.STABLE) - ShellRunner.run("sync") + Shell.run("sync") def test_packages(self): tgz_file = "/tmp/tmp.tgz" tgz_sha_file = "/tmp/tmp.tgz.sha512" - ShellRunner.run( - f"curl -o {tgz_file} -f0 {self.repo_url}/stable/clickhouse-client-{self.version}-arm64.tgz" + cmd = f"curl -o {tgz_file} -f0 {self.repo_url}/stable/clickhouse-client-{self.version}-arm64.tgz" + Shell.run( + cmd, + check=True, ) - ShellRunner.run( - f"curl -o {tgz_sha_file} -f0 {self.repo_url}/stable/clickhouse-client-{self.version}-arm64.tgz.sha512" + Shell.run( + f"curl -o {tgz_sha_file} -f0 {self.repo_url}/stable/clickhouse-client-{self.version}-arm64.tgz.sha512", + check=True, ) - expected_checksum = ShellRunner.run(f"cut -d ' ' -f 1 {tgz_sha_file}") - actual_checksum = ShellRunner.run(f"sha512sum {tgz_file} | cut -d ' ' -f 1") + expected_checksum = Shell.run(f"cut -d ' ' -f 1 {tgz_sha_file}", check=True) + actual_checksum = Shell.run(f"sha512sum {tgz_file} | cut -d ' ' -f 1") assert ( expected_checksum == actual_checksum ), f"[{actual_checksum} != {expected_checksum}]" - ShellRunner.run("rm /tmp/tmp.tgz*") + Shell.run("rm /tmp/tmp.tgz*") + release_info = ReleaseInfo.from_file() + release_info.tgz_command = cmd + release_info.dump() def parse_args() -> argparse.Namespace: @@ -338,20 +350,26 @@ if __name__ == "__main__": """ mp = R2MountPoint(MountPointApp.S3FS, dry_run=args.dry_run) if args.export_debian: - mp.init() - DebianArtifactory(release_info, dry_run=args.dry_run).export_packages() - mp.teardown() + with ReleaseContextManager(release_progress=ReleaseProgress.EXPORT_DEB) as _: + mp.init() + DebianArtifactory(release_info, dry_run=args.dry_run).export_packages() + mp.teardown() if args.export_rpm: - mp.init() - RpmArtifactory(release_info, dry_run=args.dry_run).export_packages() - mp.teardown() + with ReleaseContextManager(release_progress=ReleaseProgress.EXPORT_RPM) as _: + mp.init() + RpmArtifactory(release_info, dry_run=args.dry_run).export_packages() + mp.teardown() if args.export_tgz: - mp.init() - TgzArtifactory(release_info, dry_run=args.dry_run).export_packages() - mp.teardown() + with ReleaseContextManager(release_progress=ReleaseProgress.EXPORT_TGZ) as _: + mp.init() + TgzArtifactory(release_info, dry_run=args.dry_run).export_packages() + mp.teardown() if args.test_debian: - DebianArtifactory(release_info, dry_run=args.dry_run).test_packages() + with ReleaseContextManager(release_progress=ReleaseProgress.TEST_DEB) as _: + DebianArtifactory(release_info, dry_run=args.dry_run).test_packages() if args.test_tgz: - TgzArtifactory(release_info, dry_run=args.dry_run).test_packages() + with ReleaseContextManager(release_progress=ReleaseProgress.TEST_TGZ) as _: + TgzArtifactory(release_info, dry_run=args.dry_run).test_packages() if args.test_rpm: - RpmArtifactory(release_info, dry_run=args.dry_run).test_packages() + with ReleaseContextManager(release_progress=ReleaseProgress.TEST_RPM) as _: + RpmArtifactory(release_info, dry_run=args.dry_run).test_packages() diff --git a/tests/ci/auto_release.py b/tests/ci/auto_release.py index 5d98d8810a4..39ab3156c80 100644 --- a/tests/ci/auto_release.py +++ b/tests/ci/auto_release.py @@ -25,11 +25,22 @@ def parse_args(): action="store_true", help="Post release branch statuses", ) + parser.add_argument( + "--post-auto-release-complete", + action="store_true", + help="Post autorelease completion status", + ) parser.add_argument( "--prepare", action="store_true", help="Prepare autorelease info", ) + parser.add_argument( + "--wf-status", + type=str, + default="", + help="overall workflow status [success|failure]", + ) return parser.parse_args(), parser @@ -180,6 +191,22 @@ def main(): title=f"Auto Release Status for {release_info.release_branch}", body=release_info.to_dict(), ) + if args.post_auto_release_complete: + assert args.wf_status, "--wf-status Required with --post-auto-release-complete" + if args.wf_status != SUCCESS: + CIBuddy(dry_run=False).post_job_error( + error_description="Autorelease workflow failed", + job_name="Autorelease", + with_instance_info=False, + with_wf_link=True, + critical=True, + ) + else: + CIBuddy(dry_run=False).post_info( + title=f"Autorelease completed", + body="", + with_wf_link=True, + ) elif args.prepare: _prepare(token=args.token or get_best_robot_token()) else: diff --git a/tests/ci/ci_buddy.py b/tests/ci/ci_buddy.py index ff8fdba2b6c..3eba5532e66 100644 --- a/tests/ci/ci_buddy.py +++ b/tests/ci/ci_buddy.py @@ -109,6 +109,7 @@ class CIBuddy: job_name: str = "", with_instance_info: bool = True, with_wf_link: bool = True, + critical: bool = False, ) -> None: instance_id, instance_type = "unknown", "unknown" if with_instance_info: @@ -116,7 +117,8 @@ class CIBuddy: instance_type = Shell.run("ec2metadata --instance-type") or instance_type if not job_name: job_name = os.getenv("CHECK_NAME", "unknown") - line_err = f":red_circle: *Error: {error_description}*\n\n" + sign = ":red_circle:" if not critical else ":black_circle:" + line_err = f"{sign} *Error: {error_description}*\n\n" line_ghr = f" *Runner:* `{instance_type}`, `{instance_id}`\n" line_job = f" *Job:* `{job_name}`\n" line_pr_ = f" *PR:* , <{self.commit_url}|{self.sha}>\n" diff --git a/tests/ci/ci_utils.py b/tests/ci/ci_utils.py index eb25a53d492..efbf014cd52 100644 --- a/tests/ci/ci_utils.py +++ b/tests/ci/ci_utils.py @@ -4,7 +4,7 @@ import subprocess import time from contextlib import contextmanager from pathlib import Path -from typing import Any, Iterator, List, Union, Optional, Sequence +from typing import Any, Iterator, List, Union, Optional, Sequence, Tuple import requests @@ -142,6 +142,16 @@ class GHActions: return False + @staticmethod + def get_pr_url_by_branch(repo, branch): + get_url_cmd = ( + f"gh pr list --repo {repo} --head {branch} --json url --jq '.[0].url'" + ) + url = Shell.run(get_url_cmd) + if not url: + print(f"ERROR: PR nor found, branch [{branch}]") + return url + class Shell: @classmethod @@ -157,7 +167,10 @@ class Shell: return res.stdout.strip() @classmethod - def run(cls, command, check=False): + def run(cls, command, check=False, dry_run=False): + if dry_run: + print(f"Dry-ryn. Would run command [{command}]") + return "" print(f"Run command [{command}]") res = "" result = subprocess.run( @@ -178,6 +191,12 @@ class Shell: assert result.returncode == 0 return res.strip() + @classmethod + def run_as_daemon(cls, command): + print(f"Run daemon command [{command}]") + subprocess.Popen(command.split(" ")) # pylint:disable=consider-using-with + return 0, "" + @classmethod def check(cls, command): result = subprocess.run( diff --git a/tests/ci/create_release.py b/tests/ci/create_release.py index 414ec8afd3e..4347cfebb54 100755 --- a/tests/ci/create_release.py +++ b/tests/ci/create_release.py @@ -2,7 +2,6 @@ import argparse import dataclasses import json import os -import subprocess from contextlib import contextmanager from copy import copy @@ -13,7 +12,7 @@ from git_helper import Git, GIT_PREFIX from ssh import SSHAgent from env_helper import GITHUB_REPOSITORY, S3_BUILDS_BUCKET from s3_helper import S3Helper -from ci_utils import Shell +from ci_utils import Shell, GHActions from ci_buddy import CIBuddy from version_helper import ( FILE_WITH_VERSION_PATH, @@ -31,32 +30,62 @@ CONTRIBUTORS_PATH = get_abs_path(GENERATED_CONTRIBUTORS) RELEASE_INFO_FILE = "/tmp/release_info.json" -class ShellRunner: +class ReleaseProgress: + STARTED = "started" + DOWNLOAD_PACKAGES = "download packages" + PUSH_RELEASE_TAG = "push release tag" + PUSH_NEW_RELEASE_BRANCH = "push new release branch" + BUMP_VERSION = "bump version" + CREATE_GH_RELEASE = "create GH release" + EXPORT_TGZ = "export TGZ packages" + EXPORT_RPM = "export RPM packages" + EXPORT_DEB = "export DEB packages" + TEST_TGZ = "test TGZ packages" + TEST_RPM = "test RPM packages" + TEST_DEB = "test DEB packages" - @classmethod - def run( - cls, command, check_retcode=True, print_output=True, async_=False, dry_run=False - ): - if dry_run: - print(f"Dry-run: Would run shell command: [{command}]") - return 0, "" - print(f"Running shell command: [{command}]") - if async_: - subprocess.Popen(command.split(" ")) # pylint:disable=consider-using-with - return 0, "" - result = subprocess.run( - command + " 2>&1", - shell=True, - stdout=subprocess.PIPE, - stderr=subprocess.PIPE, - text=True, - check=True, - ) - if print_output: - print(result.stdout) - if check_retcode: - assert result.returncode == 0, f"Return code [{result.returncode}]" - return result.returncode, result.stdout + +class ReleaseProgressDescription: + OK = "OK" + FAILED = "FAILED" + + +class ReleaseContextManager: + def __init__(self, release_progress): + self.release_progress = release_progress + self.release_info = None + + def __enter__(self): + if self.release_progress == ReleaseProgress.STARTED: + # create initial release info + self.release_info = ReleaseInfo( + release_branch="NA", + commit_sha=args.ref, + release_tag="NA", + version="NA", + codename="NA", + previous_release_tag="NA", + previous_release_sha="NA", + release_progress=ReleaseProgress.STARTED, + ).dump() + else: + # fetch release info from fs and update + self.release_info = ReleaseInfo.from_file() + assert self.release_info + assert ( + self.release_info.progress_description == ReleaseProgressDescription.OK + ), "Must be OK on the start of new context" + self.release_info.release_progress = self.release_progress + self.release_info.dump() + return self.release_info + + def __exit__(self, exc_type, exc_value, traceback): + assert self.release_info + if exc_type is not None: + self.release_info.progress_description = ReleaseProgressDescription.FAILED + else: + self.release_info.progress_description = ReleaseProgressDescription.OK + self.release_info.dump() @dataclasses.dataclass @@ -74,6 +103,10 @@ class ReleaseInfo: release_url: str = "" debian_command: str = "" rpm_command: str = "" + tgz_command: str = "" + docker_command: str = "" + release_progress: str = "" + progress_description: str = "" @staticmethod def from_file() -> "ReleaseInfo": @@ -85,9 +118,9 @@ class ReleaseInfo: print(f"Dump release info into [{RELEASE_INFO_FILE}]") with open(RELEASE_INFO_FILE, "w", encoding="utf-8") as f: print(json.dumps(dataclasses.asdict(self), indent=2), file=f) + return self - @staticmethod - def prepare(commit_ref: str, release_type: str) -> None: + def prepare(self, commit_ref: str, release_type: str) -> "ReleaseInfo": version = None release_branch = None release_tag = None @@ -97,8 +130,9 @@ class ReleaseInfo: assert release_type in ("patch", "new") if release_type == "new": # check commit_ref is right and on a right branch - ShellRunner.run( - f"git merge-base --is-ancestor origin/{commit_ref} origin/master" + Shell.run( + f"git merge-base --is-ancestor origin/{commit_ref} origin/master", + check=True, ) with checkout(commit_ref): commit_sha = Shell.run(f"git rev-parse {commit_ref}", check=True) @@ -130,10 +164,11 @@ class ReleaseInfo: version.with_description(codename) release_branch = f"{version.major}.{version.minor}" release_tag = version.describe - ShellRunner.run(f"{GIT_PREFIX} fetch origin {release_branch} --tags") + Shell.run(f"{GIT_PREFIX} fetch origin {release_branch} --tags", check=True) # check commit is right and on a right branch - ShellRunner.run( - f"git merge-base --is-ancestor {commit_ref} origin/{release_branch}" + Shell.run( + f"git merge-base --is-ancestor {commit_ref} origin/{release_branch}", + check=True, ) if version.patch == 1: expected_version = copy(version) @@ -172,21 +207,22 @@ class ReleaseInfo: and version and codename in ("lts", "stable") ) - res = ReleaseInfo( - release_branch=release_branch, - commit_sha=commit_sha, - release_tag=release_tag, - version=version.string, - codename=codename, - previous_release_tag=previous_release_tag, - previous_release_sha=previous_release_sha, - ) - res.dump() + + self.release_branch = release_branch + self.commit_sha = commit_sha + self.release_tag = release_tag + self.version = version.string + self.codename = codename + self.previous_release_tag = previous_release_tag + self.previous_release_sha = previous_release_sha + self.release_progress = ReleaseProgress.STARTED + self.progress_description = ReleaseProgressDescription.OK + return self def push_release_tag(self, dry_run: bool) -> None: if dry_run: # remove locally created tag from prev run - ShellRunner.run( + Shell.run( f"{GIT_PREFIX} tag -l | grep -q {self.release_tag} && git tag -d {self.release_tag} ||:" ) # Create release tag @@ -194,16 +230,17 @@ class ReleaseInfo: f"Create and push release tag [{self.release_tag}], commit [{self.commit_sha}]" ) tag_message = f"Release {self.release_tag}" - ShellRunner.run( - f"{GIT_PREFIX} tag -a -m '{tag_message}' {self.release_tag} {self.commit_sha}" + Shell.run( + f"{GIT_PREFIX} tag -a -m '{tag_message}' {self.release_tag} {self.commit_sha}", + check=True, ) cmd_push_tag = f"{GIT_PREFIX} push origin {self.release_tag}:{self.release_tag}" - ShellRunner.run(cmd_push_tag, dry_run=dry_run) + Shell.run(cmd_push_tag, dry_run=dry_run, check=True) @staticmethod def _create_gh_label(label: str, color_hex: str, dry_run: bool) -> None: cmd = f"gh api repos/{GITHUB_REPOSITORY}/labels -f name={label} -f color={color_hex}" - ShellRunner.run(cmd, dry_run=dry_run) + Shell.run(cmd, dry_run=dry_run, check=True) def push_new_release_branch(self, dry_run: bool) -> None: assert ( @@ -220,8 +257,8 @@ class ReleaseInfo: ), f"Unexpected current version in git, must precede [{self.version}] by one step, actual [{version.string}]" if dry_run: # remove locally created branch from prev run - ShellRunner.run( - f"{GIT_PREFIX} branch -l | grep -q {new_release_branch} && git branch -d {new_release_branch} ||:" + Shell.run( + f"{GIT_PREFIX} branch -l | grep -q {new_release_branch} && git branch -d {new_release_branch}" ) print( f"Create and push new release branch [{new_release_branch}], commit [{self.commit_sha}]" @@ -234,7 +271,7 @@ class ReleaseInfo: cmd_push_branch = ( f"{GIT_PREFIX} push --set-upstream origin {new_release_branch}" ) - ShellRunner.run(cmd_push_branch, dry_run=dry_run) + Shell.run(cmd_push_branch, dry_run=dry_run, check=True) print("Create and push backport tags for new release branch") ReleaseInfo._create_gh_label( @@ -243,12 +280,13 @@ class ReleaseInfo: ReleaseInfo._create_gh_label( f"v{new_release_branch}-affected", "c2bfff", dry_run=dry_run ) - ShellRunner.run( + Shell.run( f"""gh pr create --repo {GITHUB_REPOSITORY} --title 'Release pull request for branch {new_release_branch}' --head {new_release_branch} {pr_labels} --body 'This PullRequest is a part of ClickHouse release cycle. It is used by CI system only. Do not perform any changes with it.' """, dry_run=dry_run, + check=True, ) def update_version_and_contributors_list(self, dry_run: bool) -> None: @@ -274,31 +312,34 @@ class ReleaseInfo: body_file = get_abs_path(".github/PULL_REQUEST_TEMPLATE.md") actor = os.getenv("GITHUB_ACTOR", "") or "me" cmd_create_pr = f"gh pr create --repo {GITHUB_REPOSITORY} --title 'Update version after release' --head {branch_upd_version_contributors} --base {self.release_branch} --body-file '{body_file} --label 'do not test' --assignee @{actor}" - ShellRunner.run(cmd_commit_version_upd, dry_run=dry_run) - ShellRunner.run(cmd_push_branch, dry_run=dry_run) - ShellRunner.run(cmd_create_pr, dry_run=dry_run) + Shell.run(cmd_commit_version_upd, check=True, dry_run=dry_run) + Shell.run(cmd_push_branch, check=True, dry_run=dry_run) + Shell.run(cmd_create_pr, check=True, dry_run=dry_run) if dry_run: - ShellRunner.run( - f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'" - ) - ShellRunner.run( + Shell.run(f"{GIT_PREFIX} diff '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'") + Shell.run( f"{GIT_PREFIX} checkout '{CMAKE_PATH}' '{CONTRIBUTORS_PATH}'" ) + self.version_bump_pr = GHActions.get_pr_url_by_branch( + repo=GITHUB_REPOSITORY, branch=branch_upd_version_contributors + ) - def update_release_info(self, dry_run: bool) -> None: + def update_release_info(self, dry_run: bool) -> "ReleaseInfo": branch = f"auto/{release_info.release_tag}" if not dry_run: - get_url_cmd = f"gh pr list --repo {GITHUB_REPOSITORY} --head {branch} --json url --jq '.[0].url'" - url = Shell.run(get_url_cmd) - if url: - print(f"Update release info with Changelog PR link [{url}]") - else: - print(f"WARNING: Changelog PR not found, branch [{branch}]") + url = GHActions.get_pr_url_by_branch(repo=GITHUB_REPOSITORY, branch=branch) else: url = "dry-run" + print(f"ChangeLog PR url [{url}]") self.changelog_pr = url + print(f"Release url [{url}]") + self.release_url = ( + f"https://github.com/{GITHUB_REPOSITORY}/releases/tag/{self.release_tag}" + ) + self.docker_command = f"docker run --rm clickhouse/clickhouse:{self.release_branch} clickhouse --version" self.dump() + return self def create_gh_release(self, packages_files: List[str], dry_run: bool) -> None: repo = os.getenv("GITHUB_REPOSITORY") @@ -376,7 +417,7 @@ class PackageDownloader: self.macos_package_files = ["clickhouse-macos", "clickhouse-macos-aarch64"] self.file_to_type = {} - ShellRunner.run(f"mkdir -p {self.LOCAL_DIR}") + Shell.run(f"mkdir -p {self.LOCAL_DIR}") for package_type in self.PACKAGE_TYPES: for package in self.package_names: @@ -426,7 +467,7 @@ class PackageDownloader: return res def run(self): - ShellRunner.run(f"rm -rf {self.LOCAL_DIR}/*") + Shell.run(f"rm -rf {self.LOCAL_DIR}/*") for package_file in ( self.deb_package_files + self.rpm_package_files + self.tgz_package_files ): @@ -499,6 +540,37 @@ class PackageDownloader: return True +@contextmanager +def checkout(ref: str) -> Iterator[None]: + orig_ref = Shell.run(f"{GIT_PREFIX} symbolic-ref --short HEAD", check=True) + rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}" + assert orig_ref + if ref not in (orig_ref,): + Shell.run(f"{GIT_PREFIX} checkout {ref}") + try: + yield + except (Exception, KeyboardInterrupt) as e: + print(f"ERROR: Exception [{e}]") + Shell.run(rollback_cmd) + raise + Shell.run(rollback_cmd) + + +@contextmanager +def checkout_new(ref: str) -> Iterator[None]: + orig_ref = Shell.run(f"{GIT_PREFIX} symbolic-ref --short HEAD", check=True) + rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}" + assert orig_ref + Shell.run(f"{GIT_PREFIX} checkout -b {ref}", check=True) + try: + yield + except (Exception, KeyboardInterrupt) as e: + print(f"ERROR: Exception [{e}]") + Shell.run(rollback_cmd) + raise + Shell.run(rollback_cmd) + + def parse_args() -> argparse.Namespace: parser = argparse.ArgumentParser( formatter_class=argparse.ArgumentDefaultsHelpFormatter, @@ -556,44 +628,26 @@ def parse_args() -> argparse.Namespace: action="store_true", help="do not make any actual changes in the repo, just show what will be done", ) - + parser.add_argument( + "--set-progress-started", + action="store_true", + help="Set new progress step, --progress must be set", + ) + parser.add_argument( + "--progress", + type=str, + help="Progress step name, see @ReleaseProgress", + ) + parser.add_argument( + "--set-progress-completed", + action="store_true", + help="Set current progress step to OK (completed)", + ) return parser.parse_args() -@contextmanager -def checkout(ref: str) -> Iterator[None]: - _, orig_ref = ShellRunner.run(f"{GIT_PREFIX} symbolic-ref --short HEAD") - rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}" - assert orig_ref - if ref not in (orig_ref,): - Shell.run(f"{GIT_PREFIX} checkout {ref}") - try: - yield - except (Exception, KeyboardInterrupt) as e: - print(f"ERROR: Exception [{e}]") - ShellRunner.run(rollback_cmd) - raise - ShellRunner.run(rollback_cmd) - - -@contextmanager -def checkout_new(ref: str) -> Iterator[None]: - _, orig_ref = ShellRunner.run(f"{GIT_PREFIX} symbolic-ref --short HEAD") - rollback_cmd = f"{GIT_PREFIX} checkout {orig_ref}" - assert orig_ref - ShellRunner.run(f"{GIT_PREFIX} checkout -b {ref}") - try: - yield - except (Exception, KeyboardInterrupt) as e: - print(f"ERROR: Exception [{e}]") - ShellRunner.run(rollback_cmd) - raise - ShellRunner.run(rollback_cmd) - - if __name__ == "__main__": args = parse_args() - assert args.dry_run # prepare ssh for git if needed _ssh_agent = None @@ -605,35 +659,56 @@ if __name__ == "__main__": _ssh_agent.print_keys() if args.prepare_release_info: - assert ( - args.ref and args.release_type - ), "--ref and --release-type must be provided with --prepare-release-info" - ReleaseInfo.prepare(commit_ref=args.ref, release_type=args.release_type) - if args.push_release_tag: - release_info = ReleaseInfo.from_file() - release_info.push_release_tag(dry_run=args.dry_run) - if args.push_new_release_branch: - release_info = ReleaseInfo.from_file() - release_info.push_new_release_branch(dry_run=args.dry_run) - if args.create_bump_version_pr: - # TODO: store link to PR in release info - release_info = ReleaseInfo.from_file() - release_info.update_version_and_contributors_list(dry_run=args.dry_run) + with ReleaseContextManager( + release_progress=ReleaseProgress.STARTED + ) as release_info: + assert ( + args.ref and args.release_type + ), "--ref and --release-type must be provided with --prepare-release-info" + release_info.prepare(commit_ref=args.ref, release_type=args.release_type) + if args.download_packages: - release_info = ReleaseInfo.from_file() - p = PackageDownloader( - release=release_info.release_branch, - commit_sha=release_info.commit_sha, - version=release_info.version, - ) - p.run() + with ReleaseContextManager( + release_progress=ReleaseProgress.DOWNLOAD_PACKAGES + ) as release_info: + p = PackageDownloader( + release=release_info.release_branch, + commit_sha=release_info.commit_sha, + version=release_info.version, + ) + p.run() + + if args.push_release_tag: + with ReleaseContextManager( + release_progress=ReleaseProgress.PUSH_RELEASE_TAG + ) as release_info: + release_info.push_release_tag(dry_run=args.dry_run) + + if args.push_new_release_branch: + with ReleaseContextManager( + release_progress=ReleaseProgress.PUSH_NEW_RELEASE_BRANCH + ) as release_info: + release_info.push_new_release_branch(dry_run=args.dry_run) + + if args.create_bump_version_pr: + with ReleaseContextManager( + release_progress=ReleaseProgress.BUMP_VERSION + ) as release_info: + release_info.update_version_and_contributors_list(dry_run=args.dry_run) + if args.create_gh_release: - release_info = ReleaseInfo.from_file() - p = PackageDownloader( - release=release_info.release_branch, - commit_sha=release_info.commit_sha, - version=release_info.version, - ) + with ReleaseContextManager( + release_progress=ReleaseProgress.CREATE_GH_RELEASE + ) as release_info: + p = PackageDownloader( + release=release_info.release_branch, + commit_sha=release_info.commit_sha, + version=release_info.version, + ) + release_info.create_gh_release( + packages_files=p.get_all_packages_files(), dry_run=args.dry_run + ) + if args.post_status: release_info = ReleaseInfo.from_file() release_info.update_release_info(dry_run=args.dry_run) @@ -646,6 +721,21 @@ if __name__ == "__main__": f"Failed to issue new release", dataclasses.asdict(release_info) ) + if args.set_progress_started: + ri = ReleaseInfo.from_file() + ri.release_progress = args.progress + ri.progress_description = ReleaseProgressDescription.FAILED + ri.dump() + assert args.progress, "Progress step name must be provided" + + if args.set_progress_completed: + ri = ReleaseInfo.from_file() + assert ( + ri.progress_description == ReleaseProgressDescription.FAILED + ), "Must be FAILED before set to OK" + ri.progress_description = ReleaseProgressDescription.OK + ri.dump() + # tear down ssh if _ssh_agent and _key_pub: _ssh_agent.remove(_key_pub) From f1021b70f7f79fc6f921989573435ab5df406bdc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 07:05:09 +0200 Subject: [PATCH 14/26] Better diagnostics for test trace_events_stress --- src/Common/TraceSender.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Common/TraceSender.cpp b/src/Common/TraceSender.cpp index 91d07367a82..064da1b3d76 100644 --- a/src/Common/TraceSender.cpp +++ b/src/Common/TraceSender.cpp @@ -23,8 +23,15 @@ namespace DB LazyPipeFDs TraceSender::pipe; +static thread_local bool inside_send = false; void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Extras extras) { + DENY_ALLOCATIONS_IN_SCOPE; + + if (unlikely(inside_send)) + abort(); /// The method shouldn't be called recursively or throw exceptions. + inside_send = true; + constexpr size_t buf_size = sizeof(char) /// TraceCollector stop flag + sizeof(UInt8) /// String size + QUERY_ID_MAX_LEN /// Maximum query_id length @@ -80,6 +87,8 @@ void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Ext writePODBinary(extras.increment, out); out.next(); + + inside_send = false; } } From 6437088d81200909245172c441de3bf301f553b0 Mon Sep 17 00:00:00 2001 From: Mikhail Date: Sun, 21 Jul 2024 12:52:26 +0700 Subject: [PATCH 15/26] translate playground page to russian in ru locale --- docs/ru/getting-started/playground.md | 40 +++++++++++++-------------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/docs/ru/getting-started/playground.md b/docs/ru/getting-started/playground.md index c8c987eec9e..eb990c6071e 100644 --- a/docs/ru/getting-started/playground.md +++ b/docs/ru/getting-started/playground.md @@ -1,43 +1,43 @@ --- slug: /ru/getting-started/playground sidebar_position: 14 -sidebar_label: Playground +sidebar_label: Песочница --- -# ClickHouse Playground {#clickhouse-playground} +# Песочница ClickHouse {#clickhouse-playground} -[ClickHouse Playground](https://play.clickhouse.com/play?user=play) allows people to experiment with ClickHouse by running queries instantly, without setting up their server or cluster. -Several example datasets are available in Playground. +[Песочница ClickHouse](https://play.clickhouse.com/play?user=play) позволяет пользователям экспериментировать с ClickHouse, выполняя запросы мгновенно, без необходимости настройки сервера или кластера. +В Песочнице доступны несколько примеров наборов данных. -You can make queries to Playground using any HTTP client, for example [curl](https://curl.haxx.se) or [wget](https://www.gnu.org/software/wget/), or set up a connection using [JDBC](../interfaces/jdbc.md) or [ODBC](../interfaces/odbc.md) drivers. More information about software products that support ClickHouse is available [here](../interfaces/index.md). +Вы можете выполнять запросы к Песочнице, используя любой HTTP-клиент, например [curl](https://curl.haxx.se) или [wget](https://www.gnu.org/software/wget/), или настроить соединение, используя драйверы [JDBC](../interfaces/jdbc.md) или [ODBC](../interfaces/odbc.md). Дополнительную информацию о программных продуктах, поддерживающих ClickHouse, можно найти [здесь](../interfaces/index.md). -## Credentials {#credentials} +## Учетные данные {#credentials} -| Parameter | Value | +| Параметр | Значение | |:--------------------|:-----------------------------------| -| HTTPS endpoint | `https://play.clickhouse.com:443/` | -| Native TCP endpoint | `play.clickhouse.com:9440` | -| User | `explorer` or `play` | -| Password | (empty) | +| HTTPS-адрес | `https://play.clickhouse.com:443/` | +| TCP-адрес | `play.clickhouse.com:9440` | +| Пользователь | `explorer` или `play` | +| Пароль | (пусто) | -## Limitations {#limitations} +## Ограничения {#limitations} -The queries are executed as a read-only user. It implies some limitations: +Запросы выполняются от имени пользователя с правами только на чтение. Это предполагает некоторые ограничения: -- DDL queries are not allowed -- INSERT queries are not allowed +- DDL-запросы не разрешены +- INSERT-запросы не разрешены -The service also have quotas on its usage. +Сервис также имеет квоты на использование. -## Examples {#examples} +## Примеры {#examples} -HTTPS endpoint example with `curl`: +Пример использования HTTPS-адреса с `curl`: -``` bash +```bash curl "https://play.clickhouse.com/?user=explorer" --data-binary "SELECT 'Play ClickHouse'" ``` -TCP endpoint example with [CLI](../interfaces/cli.md): +Пример использования TCP-адреса с [CLI](../interfaces/cli.md): ``` bash clickhouse client --secure --host play.clickhouse.com --user explorer From e74892bfaf4aa98654a6253a34bbeb3f1740de41 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 08:21:09 +0200 Subject: [PATCH 16/26] Update playground.md --- docs/ru/getting-started/playground.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/ru/getting-started/playground.md b/docs/ru/getting-started/playground.md index eb990c6071e..a2d5498fb9a 100644 --- a/docs/ru/getting-started/playground.md +++ b/docs/ru/getting-started/playground.md @@ -1,15 +1,15 @@ --- slug: /ru/getting-started/playground sidebar_position: 14 -sidebar_label: Песочница +sidebar_label: Playground --- -# Песочница ClickHouse {#clickhouse-playground} +# ClickHouse Playground {#clickhouse-playground} -[Песочница ClickHouse](https://play.clickhouse.com/play?user=play) позволяет пользователям экспериментировать с ClickHouse, выполняя запросы мгновенно, без необходимости настройки сервера или кластера. -В Песочнице доступны несколько примеров наборов данных. +[ClickHouse Playground](https://play.clickhouse.com/play?user=play) позволяет пользователям экспериментировать с ClickHouse, выполняя запросы мгновенно, без необходимости настройки сервера или кластера. +В Playground доступны несколько примеров наборов данных. -Вы можете выполнять запросы к Песочнице, используя любой HTTP-клиент, например [curl](https://curl.haxx.se) или [wget](https://www.gnu.org/software/wget/), или настроить соединение, используя драйверы [JDBC](../interfaces/jdbc.md) или [ODBC](../interfaces/odbc.md). Дополнительную информацию о программных продуктах, поддерживающих ClickHouse, можно найти [здесь](../interfaces/index.md). +Вы можете выполнять запросы к Playground, используя любой HTTP-клиент, например [curl](https://curl.haxx.se) или [wget](https://www.gnu.org/software/wget/), или настроить соединение, используя драйверы [JDBC](../interfaces/jdbc.md) или [ODBC](../interfaces/odbc.md). Дополнительную информацию о программных продуктах, поддерживающих ClickHouse, можно найти [здесь](../interfaces/index.md). ## Учетные данные {#credentials} From a371557f07ab97b88da40e6b790a5d361f437a47 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 09:06:45 +0200 Subject: [PATCH 17/26] Make test `00997_set_index_array` lighter --- tests/queries/0_stateless/00997_set_index_array.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/00997_set_index_array.sql b/tests/queries/0_stateless/00997_set_index_array.sql index d6d27f5a6a0..ed972d1a545 100644 --- a/tests/queries/0_stateless/00997_set_index_array.sql +++ b/tests/queries/0_stateless/00997_set_index_array.sql @@ -12,10 +12,10 @@ ORDER BY (primary_key); INSERT INTO set_array select - toString(intDiv(number, 1000000)) as primary_key, + toString(intDiv(number, 100000)) as primary_key, array(number) as index_array from system.numbers -limit 10000000; +limit 1000000; OPTIMIZE TABLE set_array FINAL; From ea83f89374f723d1d53dca64ec58e8fd993f013d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 10:39:03 +0200 Subject: [PATCH 18/26] Fix inconsistent formatting of lambda functions inside composite types --- src/Parsers/ASTFunction.cpp | 8 ++++---- ...ite_expressions_lambda_consistent_formatting.reference | 0 ...composite_expressions_lambda_consistent_formatting.sql | 6 ++++++ 3 files changed, 10 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.reference create mode 100644 tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index f39229d7566..b04ec1c22b2 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include @@ -19,9 +18,6 @@ #include #include #include -#include - -#include using namespace std::literals; @@ -632,6 +628,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format settings.ostr << ", "; if (arguments->children[i]->as()) settings.ostr << "SETTINGS "; + nested_dont_need_parens.list_element_index = i; arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens); } settings.ostr << (settings.hilite ? hilite_operator : "") << ']' << (settings.hilite ? hilite_none : ""); @@ -642,12 +639,14 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format { settings.ostr << (settings.hilite ? hilite_operator : "") << ((frame.need_parens && !alias.empty()) ? "tuple" : "") << '(' << (settings.hilite ? hilite_none : ""); + for (size_t i = 0; i < arguments->children.size(); ++i) { if (i != 0) settings.ostr << ", "; if (arguments->children[i]->as()) settings.ostr << "SETTINGS "; + nested_dont_need_parens.list_element_index = i; arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens); } settings.ostr << (settings.hilite ? hilite_operator : "") << ')' << (settings.hilite ? hilite_none : ""); @@ -663,6 +662,7 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format settings.ostr << ", "; if (arguments->children[i]->as()) settings.ostr << "SETTINGS "; + nested_dont_need_parens.list_element_index = i; arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens); } settings.ostr << (settings.hilite ? hilite_operator : "") << ')' << (settings.hilite ? hilite_none : ""); diff --git a/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.reference b/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql b/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql new file mode 100644 index 00000000000..42c823cf476 --- /dev/null +++ b/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql @@ -0,0 +1,6 @@ +SELECT [1, (x -> 1)]; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT (1, (x -> 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT map(1, (x -> 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT [1, lambda(x, 1)]; -- { serverError UNKNOWN_IDENTIFIER } +SELECT (1, lambda(x, 1)); -- { serverError UNKNOWN_IDENTIFIER } +SELECT map(1, lambda(x, 1)); -- { serverError UNKNOWN_IDENTIFIER } From 433ac55d13276e7a0b26ded07f27f17726703dab Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sun, 21 Jul 2024 11:41:42 +0200 Subject: [PATCH 19/26] Correctly handle failure --- src/Common/Allocator.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/Allocator.cpp b/src/Common/Allocator.cpp index 7f2241ab4c0..1b43e746c69 100644 --- a/src/Common/Allocator.cpp +++ b/src/Common/Allocator.cpp @@ -206,7 +206,7 @@ void * Allocator::realloc(void * buf, size_t old_size, } else { - [[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size); + [[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(new_size); ProfileEvents::increment(ProfileEvents::GWPAsanAllocateFailed); } } @@ -239,7 +239,7 @@ void * Allocator::realloc(void * buf, size_t old_size, void * new_buf = ::realloc(buf, new_size); if (nullptr == new_buf) { - [[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(old_size); + [[maybe_unused]] auto trace_free = CurrentMemoryTracker::free(new_size); throw DB::ErrnoException( DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, "Allocator: Cannot realloc from {} to {}", From 08353419d0798f209627f16f840f4d101227431d Mon Sep 17 00:00:00 2001 From: Max K Date: Sun, 21 Jul 2024 11:50:15 +0200 Subject: [PATCH 20/26] CI: Never await on BuildReport job, Skip BuildReport if no builds in workflow --- tests/ci/ci_cache.py | 27 ++++++++++---- tests/ci/ci_definitions.py | 3 ++ tests/ci/merge_pr.py | 5 ++- tests/ci/test_ci_config.py | 74 ++++++++++++++++++++++++++++++++++---- 4 files changed, 96 insertions(+), 13 deletions(-) diff --git a/tests/ci/ci_cache.py b/tests/ci/ci_cache.py index cfefb954fcd..16b6eac1ecb 100644 --- a/tests/ci/ci_cache.py +++ b/tests/ci/ci_cache.py @@ -638,7 +638,7 @@ class CiCache: pushes pending records for all jobs that supposed to be run """ for job, job_config in self.jobs_to_do.items(): - if not job_config.has_digest(): + if not job_config.has_digest() or job_config.disable_await: continue pending_state = PendingState(time.time(), run_url=GITHUB_RUN_URL) assert job_config.batches @@ -708,7 +708,7 @@ class CiCache: Filter is to be applied in PRs to remove jobs that are not affected by the change :return: """ - remove_from_to_do = [] + remove_from_workflow = [] required_builds = [] has_test_jobs_to_skip = False for job_name, job_config in self.jobs_to_do.items(): @@ -723,26 +723,41 @@ class CiCache: job=reference_name, job_config=reference_config, ): - remove_from_to_do.append(job_name) + remove_from_workflow.append(job_name) has_test_jobs_to_skip = True else: required_builds += ( job_config.required_builds if job_config.required_builds else [] ) if has_test_jobs_to_skip: - # If there are tests to skip, it means build digest has not been changed. + # If there are tests to skip, it means builds are not affected as well. # No need to test builds. Let's keep all builds required for test jobs and skip the others for job_name, job_config in self.jobs_to_do.items(): if CI.is_build_job(job_name): if job_name not in required_builds: - remove_from_to_do.append(job_name) + remove_from_workflow.append(job_name) - for job in remove_from_to_do: + for job in remove_from_workflow: print(f"Filter job [{job}] - not affected by the change") if job in self.jobs_to_do: del self.jobs_to_do[job] if job in self.jobs_to_wait: del self.jobs_to_wait[job] + if job in self.jobs_to_skip: + self.jobs_to_skip.remove(job) + + # special handling for the special job: BUILD_CHECK + has_builds = False + for job in list(self.jobs_to_do) + self.jobs_to_skip: + if CI.is_build_job(job): + has_builds = True + break + if not has_builds: + if CI.JobNames.BUILD_CHECK in self.jobs_to_do: + print( + f"Filter job [{CI.JobNames.BUILD_CHECK}] - no builds are required in the workflow" + ) + del self.jobs_to_do[CI.JobNames.BUILD_CHECK] def await_pending_jobs(self, is_release: bool, dry_run: bool = False) -> None: """ diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index 4c78efd39a2..a8d9793f1d3 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -351,6 +351,8 @@ class JobConfig: run_by_label: str = "" # to run always regardless of the job digest or/and label run_always: bool = False + # disables CI await for a given job + disable_await: bool = False # if the job needs to be run on the release branch, including master (building packages, docker server). # NOTE: Subsequent runs on the same branch with the similar digest are still considered skip-able. required_on_release_branch: bool = False @@ -395,6 +397,7 @@ class CommonJobConfigs: ], ), runner_type=Runners.STYLE_CHECKER_ARM, + disable_await=True, ) COMPATIBILITY_TEST = JobConfig( job_name_keyword="compatibility", diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 4d8facafb84..13c7537a84b 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -254,11 +254,14 @@ def main(): statuses = get_commit_filtered_statuses(commit) has_failed_statuses = False + has_native_failed_status = False for status in statuses: print(f"Check status [{status.context}], [{status.state}]") if CI.is_required(status.context) and status.state != SUCCESS: print(f"WARNING: Failed status [{status.context}], [{status.state}]") has_failed_statuses = True + if status.context != CI.StatusNames.SYNC: + has_native_failed_status = True if args.wf_status == SUCCESS or has_failed_statuses: # set Mergeable check if workflow is successful (green) @@ -280,7 +283,7 @@ def main(): print( "Workflow failed but no failed statuses found (died runner?) - cannot set Mergeable Check status" ) - if args.wf_status == SUCCESS and not has_failed_statuses: + if args.wf_status == SUCCESS and not has_native_failed_status: sys.exit(0) else: sys.exit(1) diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 12e863c4d8d..10867ea1444 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -587,11 +587,11 @@ class TestCIConfig(unittest.TestCase): for job, job_config in ci_cache.jobs_to_do.items(): if job in MOCK_AFFECTED_JOBS: MOCK_REQUIRED_BUILDS += job_config.required_builds - elif job not in MOCK_AFFECTED_JOBS: + elif job not in MOCK_AFFECTED_JOBS and not job_config.disable_await: ci_cache.jobs_to_wait[job] = job_config for job, job_config in ci_cache.jobs_to_do.items(): - if job_config.reference_job_name: + if job_config.reference_job_name or job_config.disable_await: # jobs with reference_job_name in config are not supposed to have records in the cache - continue continue if job in MOCK_AFFECTED_JOBS: @@ -624,11 +624,73 @@ class TestCIConfig(unittest.TestCase): + MOCK_AFFECTED_JOBS + MOCK_REQUIRED_BUILDS ) + self.assertTrue(CI.JobNames.BUILD_CHECK not in ci_cache.jobs_to_wait, "We must never await on Builds Report") self.assertCountEqual( list(ci_cache.jobs_to_wait), - [ - CI.JobNames.BUILD_CHECK, - ] - + MOCK_REQUIRED_BUILDS, + MOCK_REQUIRED_BUILDS, + ) + self.assertCountEqual(list(ci_cache.jobs_to_do), expected_to_do) + + def test_ci_py_filters_not_affected_jobs_in_prs_no_builds(self): + """ + checks ci.py filters not affected jobs in PRs, no builds required + """ + settings = CiSettings() + settings.no_ci_cache = True + pr_info = PRInfo(github_event=_TEST_EVENT_JSON) + pr_info.event_type = EventType.PULL_REQUEST + pr_info.number = 123 + assert pr_info.is_pr + ci_cache = CIPY._configure_jobs( + S3Helper(), pr_info, settings, skip_jobs=False, dry_run=True + ) + self.assertTrue(not ci_cache.jobs_to_skip, "Must be no jobs in skip list") + assert not ci_cache.jobs_to_wait + assert not ci_cache.jobs_to_skip + + MOCK_AFFECTED_JOBS = [ + CI.JobNames.DOCS_CHECK, + ] + MOCK_REQUIRED_BUILDS = [] + + # pretend there are pending jobs that we need to wait + for job, job_config in ci_cache.jobs_to_do.items(): + if job in MOCK_AFFECTED_JOBS: + if job_config.required_builds: + MOCK_REQUIRED_BUILDS += job_config.required_builds + elif job not in MOCK_AFFECTED_JOBS and not job_config.disable_await: + ci_cache.jobs_to_wait[job] = job_config + + for job, job_config in ci_cache.jobs_to_do.items(): + if job_config.reference_job_name or job_config.disable_await: + # jobs with reference_job_name in config are not supposed to have records in the cache - continue + continue + if job in MOCK_AFFECTED_JOBS: + continue + for batch in range(job_config.num_batches): + # add any record into cache + record = CiCache.Record( + record_type=random.choice( + [ + CiCache.RecordType.FAILED, + CiCache.RecordType.PENDING, + CiCache.RecordType.SUCCESSFUL, + ] + ), + job_name=job, + job_digest=ci_cache.job_digests[job], + batch=batch, + num_batches=job_config.num_batches, + release_branch=True, + ) + for record_t_, records_ in ci_cache.records.items(): + if record_t_.value == CiCache.RecordType.FAILED.value: + records_[record.to_str_key()] = record + + ci_cache.filter_out_not_affected_jobs() + expected_to_do = MOCK_AFFECTED_JOBS + MOCK_REQUIRED_BUILDS + self.assertCountEqual( + list(ci_cache.jobs_to_wait), + MOCK_REQUIRED_BUILDS, ) self.assertCountEqual(list(ci_cache.jobs_to_do), expected_to_do) From c61581ca69757ce9c1d15b6ccc139d48d27f5b07 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 12:06:53 +0200 Subject: [PATCH 21/26] Fix error --- src/Common/TraceSender.cpp | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Common/TraceSender.cpp b/src/Common/TraceSender.cpp index 064da1b3d76..f1adf7c516a 100644 --- a/src/Common/TraceSender.cpp +++ b/src/Common/TraceSender.cpp @@ -26,11 +26,16 @@ LazyPipeFDs TraceSender::pipe; static thread_local bool inside_send = false; void TraceSender::send(TraceType trace_type, const StackTrace & stack_trace, Extras extras) { - DENY_ALLOCATIONS_IN_SCOPE; - + /** The method shouldn't be called recursively or throw exceptions. + * There are several reasons: + * - avoid infinite recursion when some of subsequent functions invoke tracing; + * - avoid inconsistent writes if the method was interrupted by a signal handler in the middle of writing, + * and then another tracing is invoked (e.g., from query profiler). + */ if (unlikely(inside_send)) - abort(); /// The method shouldn't be called recursively or throw exceptions. + return; inside_send = true; + DENY_ALLOCATIONS_IN_SCOPE; constexpr size_t buf_size = sizeof(char) /// TraceCollector stop flag + sizeof(UInt8) /// String size From 02bf9e4243c329611fc0be43432d3f9c290512d5 Mon Sep 17 00:00:00 2001 From: Max K Date: Sun, 21 Jul 2024 12:46:58 +0200 Subject: [PATCH 22/26] push pending records before await call --- tests/ci/ci.py | 5 +- tests/ci/ci_cache.py | 93 +++++++++++++++++++++++++++++++++++++- tests/ci/test_ci_config.py | 5 +- 3 files changed, 99 insertions(+), 4 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index ff31d1ce489..f6bec5304a0 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1110,13 +1110,14 @@ def main() -> int: ci_cache.print_status() if IS_CI and not pr_info.is_merge_queue: - # wait for pending jobs to be finished, await_jobs is a long blocking call - ci_cache.await_pending_jobs(pr_info.is_release) if pr_info.is_release: print("Release/master: CI Cache add pending records for all todo jobs") ci_cache.push_pending_all(pr_info.is_release) + # wait for pending jobs to be finished, await_jobs is a long blocking call + ci_cache.await_pending_jobs(pr_info.is_release) + # conclude results result["git_ref"] = git_ref result["version"] = version diff --git a/tests/ci/ci_cache.py b/tests/ci/ci_cache.py index 16b6eac1ecb..85eabb84f9f 100644 --- a/tests/ci/ci_cache.py +++ b/tests/ci/ci_cache.py @@ -638,7 +638,14 @@ class CiCache: pushes pending records for all jobs that supposed to be run """ for job, job_config in self.jobs_to_do.items(): - if not job_config.has_digest() or job_config.disable_await: + if ( + job in self.jobs_to_wait + or not job_config.has_digest() + or job_config.disable_await + ): + # 1. "job in self.jobs_to_wait" - this job already has a pending record in cache + # 2. "not job_config.has_digest()" - cache is not used for these jobs + # 3. "job_config.disable_await" - await is explicitly disabled continue pending_state = PendingState(time.time(), run_url=GITHUB_RUN_URL) assert job_config.batches @@ -899,3 +906,87 @@ class CiCache: self.jobs_to_wait[job] = job_config return self + + +if __name__ == "__main__": + # for testing + job_digest = { + "package_release": "bbbd3519d1", + "package_aarch64": "bbbd3519d1", + "package_asan": "bbbd3519d1", + "package_ubsan": "bbbd3519d1", + "package_tsan": "bbbd3519d1", + "package_msan": "bbbd3519d1", + "package_debug": "bbbd3519d1", + "package_release_coverage": "bbbd3519d1", + "binary_release": "bbbd3519d1", + "binary_tidy": "bbbd3519d1", + "binary_darwin": "bbbd3519d1", + "binary_aarch64": "bbbd3519d1", + "binary_aarch64_v80compat": "bbbd3519d1", + "binary_freebsd": "bbbd3519d1", + "binary_darwin_aarch64": "bbbd3519d1", + "binary_ppc64le": "bbbd3519d1", + "binary_amd64_compat": "bbbd3519d1", + "binary_amd64_musl": "bbbd3519d1", + "binary_riscv64": "bbbd3519d1", + "binary_s390x": "bbbd3519d1", + "binary_loongarch64": "bbbd3519d1", + "Builds": "f5dffeecb8", + "Install packages (release)": "ba0c89660e", + "Install packages (aarch64)": "ba0c89660e", + "Stateful tests (asan)": "32a9a1aba9", + "Stateful tests (tsan)": "32a9a1aba9", + "Stateful tests (msan)": "32a9a1aba9", + "Stateful tests (ubsan)": "32a9a1aba9", + "Stateful tests (debug)": "32a9a1aba9", + "Stateful tests (release)": "32a9a1aba9", + "Stateful tests (coverage)": "32a9a1aba9", + "Stateful tests (aarch64)": "32a9a1aba9", + "Stateful tests (release, ParallelReplicas)": "32a9a1aba9", + "Stateful tests (debug, ParallelReplicas)": "32a9a1aba9", + "Stateless tests (asan)": "deb6778b88", + "Stateless tests (tsan)": "deb6778b88", + "Stateless tests (msan)": "deb6778b88", + "Stateless tests (ubsan)": "deb6778b88", + "Stateless tests (debug)": "deb6778b88", + "Stateless tests (release)": "deb6778b88", + "Stateless tests (coverage)": "deb6778b88", + "Stateless tests (aarch64)": "deb6778b88", + "Stateless tests (release, old analyzer, s3, DatabaseReplicated)": "deb6778b88", + "Stateless tests (debug, s3 storage)": "deb6778b88", + "Stateless tests (tsan, s3 storage)": "deb6778b88", + "Stress test (debug)": "aa298abf10", + "Stress test (tsan)": "aa298abf10", + "Upgrade check (debug)": "5ce4d3ee02", + "Integration tests (asan, old analyzer)": "42e58be3aa", + "Integration tests (tsan)": "42e58be3aa", + "Integration tests (aarch64)": "42e58be3aa", + "Integration tests flaky check (asan)": "42e58be3aa", + "Compatibility check (release)": "ecb69d8c4b", + "Compatibility check (aarch64)": "ecb69d8c4b", + "Unit tests (release)": "09d00b702e", + "Unit tests (asan)": "09d00b702e", + "Unit tests (msan)": "09d00b702e", + "Unit tests (tsan)": "09d00b702e", + "Unit tests (ubsan)": "09d00b702e", + "AST fuzzer (debug)": "c38ebf947f", + "AST fuzzer (asan)": "c38ebf947f", + "AST fuzzer (msan)": "c38ebf947f", + "AST fuzzer (tsan)": "c38ebf947f", + "AST fuzzer (ubsan)": "c38ebf947f", + "Stateless tests flaky check (asan)": "deb6778b88", + "Performance Comparison (release)": "a8a7179258", + "ClickBench (release)": "45c07c4aa6", + "ClickBench (aarch64)": "45c07c4aa6", + "Docker server image": "6a24d5b187", + "Docker keeper image": "6a24d5b187", + "Docs check": "4764154c62", + "Fast test": "cb269133f2", + "Style check": "ffffffffff", + "Stateful tests (ubsan, ParallelReplicas)": "32a9a1aba9", + "Stress test (msan)": "aa298abf10", + "Upgrade check (asan)": "5ce4d3ee02", + } + ci_cache = CiCache(job_digests=job_digest, cache_enabled=True, s3=S3Helper()) + ci_cache.update() diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 10867ea1444..04fd44a87e9 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -624,7 +624,10 @@ class TestCIConfig(unittest.TestCase): + MOCK_AFFECTED_JOBS + MOCK_REQUIRED_BUILDS ) - self.assertTrue(CI.JobNames.BUILD_CHECK not in ci_cache.jobs_to_wait, "We must never await on Builds Report") + self.assertTrue( + CI.JobNames.BUILD_CHECK not in ci_cache.jobs_to_wait, + "We must never await on Builds Report", + ) self.assertCountEqual( list(ci_cache.jobs_to_wait), MOCK_REQUIRED_BUILDS, From d83428daafc665542023fac0a9add048603ad224 Mon Sep 17 00:00:00 2001 From: Max K Date: Sun, 21 Jul 2024 14:52:33 +0200 Subject: [PATCH 23/26] fix in ci unittests --- tests/ci/test_ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index 04fd44a87e9..44142050821 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -652,7 +652,7 @@ class TestCIConfig(unittest.TestCase): assert not ci_cache.jobs_to_skip MOCK_AFFECTED_JOBS = [ - CI.JobNames.DOCS_CHECK, + CI.JobNames.FAST_TEST, ] MOCK_REQUIRED_BUILDS = [] From 20d4b16fc1d92632b714368c85ac6701723ef096 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Jul 2024 22:19:30 +0200 Subject: [PATCH 24/26] Update 03207_composite_expressions_lambda_consistent_formatting.sql --- .../03207_composite_expressions_lambda_consistent_formatting.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql b/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql index 42c823cf476..2e2f5510876 100644 --- a/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql +++ b/tests/queries/0_stateless/03207_composite_expressions_lambda_consistent_formatting.sql @@ -1,3 +1,4 @@ +SET allow_experimental_analyzer = 1; SELECT [1, (x -> 1)]; -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT (1, (x -> 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT map(1, (x -> 1)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From ed02246e0c2019f9ee661e9636f166b955a672de Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 08:46:41 +0200 Subject: [PATCH 25/26] What if I will change this test? --- tests/integration/test_ssl_cert_authentication/test.py | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 756a1e1996c..3af88759e82 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -43,15 +43,10 @@ def started_cluster(): config = """ - none - + strict {certificateFile} {privateKeyFile} {caConfig} - - - AcceptCertificateHandler - """ From 3d4604a64ad01c70c80924ea11154514bab4e424 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Jul 2024 08:47:08 +0200 Subject: [PATCH 26/26] Revert "What if I will change this test?" This reverts commit ed02246e0c2019f9ee661e9636f166b955a672de. --- tests/integration/test_ssl_cert_authentication/test.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_ssl_cert_authentication/test.py b/tests/integration/test_ssl_cert_authentication/test.py index 3af88759e82..756a1e1996c 100644 --- a/tests/integration/test_ssl_cert_authentication/test.py +++ b/tests/integration/test_ssl_cert_authentication/test.py @@ -43,10 +43,15 @@ def started_cluster(): config = """ - strict + none + {certificateFile} {privateKeyFile} {caConfig} + + + AcceptCertificateHandler + """