From c475e706d34761d7b5ff94b5f186e6f0e5479436 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 27 Jun 2023 13:20:00 +0000 Subject: [PATCH 01/22] Fix optimization to move functions before sorting. --- .../Optimizations/liftUpFunctions.cpp | 20 +++ ..._and_columns_with_same_names_bug.reference | 0 ...orting_and_columns_with_same_names_bug.sql | 133 ++++++++++++++++++ 3 files changed, 153 insertions(+) create mode 100644 tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug.reference create mode 100644 tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug.sql diff --git a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp index c3b03a5385f..b2c3f3b4a6d 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -28,6 +29,22 @@ const DB::DataStream & getChildOutputStream(DB::QueryPlan::Node & node) namespace DB::QueryPlanOptimizations { +/// This is a check that output columns with the same name have the same types. +/// This is ok to have such a situation in DAG, but not for Block. +/// TODO: we should have a different data structure for headers. +static bool areOutputsAreConvertableToBlock(const ActionsDAG::NodeRawConstPtrs & outputs) +{ + std::unordered_map name_to_type; + for (const auto & output : outputs) + { + auto [it, inserted] = name_to_type.emplace(output->result_name, output->result_type.get()); + if (!inserted && !it->second->equals(*output->result_type)) + return false; + } + + return true; +} + size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) { if (parent_node->children.size() != 1) @@ -57,6 +74,9 @@ size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan: if (unneeded_for_sorting->trivial()) return 0; + if (!areOutputsAreConvertableToBlock(needed_for_sorting->getOutputs())) + return 0; + // Sorting (parent_node) -> Expression (child_node) auto & node_with_needed = nodes.emplace_back(); std::swap(node_with_needed.children, child_node->children); diff --git a/tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug.reference b/tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug.sql b/tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug.sql new file mode 100644 index 00000000000..4a9ede36335 --- /dev/null +++ b/tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug.sql @@ -0,0 +1,133 @@ +drop table if exists test; +drop table if exists test1; + +CREATE TABLE test +( + `pt` String, + `count_distinct_exposure_uv` AggregateFunction(uniqHLL12, Int64) +) +ENGINE = AggregatingMergeTree +ORDER BY pt; + +SELECT * +FROM +( + SELECT m0.pt AS pt + ,m0.`exposure_uv` AS exposure_uv + ,round(m2.exposure_uv,4) AS exposure_uv_hb_last_value + ,if(m2.exposure_uv IS NULL OR m2.exposure_uv = 0,NULL,round((m0.exposure_uv - m2.exposure_uv) * 1.0 / m2.exposure_uv,4)) AS exposure_uv_hb_diff_percent + ,round(m1.exposure_uv,4) AS exposure_uv_tb_last_value + ,if(m1.exposure_uv IS NULL OR m1.exposure_uv = 0,NULL,round((m0.exposure_uv - m1.exposure_uv) * 1.0 / m1.exposure_uv,4)) AS exposure_uv_tb_diff_percent + FROM + ( + SELECT m0.pt AS pt + ,`exposure_uv` AS `exposure_uv` + FROM + ( + SELECT pt AS pt + ,CASE WHEN COUNT(`exposure_uv`) > 0 THEN AVG(`exposure_uv`) ELSE 0 END AS `exposure_uv` + FROM + ( + SELECT pt AS pt + ,uniqHLL12Merge(count_distinct_exposure_uv) AS `exposure_uv` + FROM test + GROUP BY pt + ) m + GROUP BY pt + ) m0 + ) m0 + LEFT JOIN + ( + SELECT m0.pt AS pt + ,`exposure_uv` AS `exposure_uv` + FROM + ( + SELECT formatDateTime(addYears(parseDateTimeBestEffort(pt),1),'%Y%m%d') AS pt + ,CASE WHEN COUNT(`exposure_uv`) > 0 THEN AVG(`exposure_uv`) ELSE 0 END AS `exposure_uv` + FROM + ( + SELECT pt AS pt + ,uniqHLL12Merge(count_distinct_exposure_uv) AS `exposure_uv` + FROM test + GROUP BY pt + ) m + GROUP BY pt + ) m0 + ) m1 + ON m0.pt = m1.pt + LEFT JOIN + ( + SELECT m0.pt AS pt + ,`exposure_uv` AS `exposure_uv` + FROM + ( + SELECT formatDateTime(addDays(toDate(parseDateTimeBestEffort(pt)),1),'%Y%m%d') AS pt + ,CASE WHEN COUNT(`exposure_uv`) > 0 THEN AVG(`exposure_uv`) ELSE 0 END AS `exposure_uv` + FROM + ( + SELECT pt AS pt + ,uniqHLL12Merge(count_distinct_exposure_uv) AS `exposure_uv` + FROM test + GROUP BY pt + ) m + GROUP BY pt + ) m0 + ) m2 + ON m0.pt = m2.pt +) c0 +ORDER BY pt ASC, exposure_uv DESC +settings join_use_nulls = 1; + +CREATE TABLE test1 +( + `pt` String, + `exposure_uv` Float64 +) +ENGINE = Memory; + +SELECT * +FROM +( + SELECT m0.pt + ,m0.exposure_uv AS exposure_uv + ,round(m2.exposure_uv,4) + FROM + ( + SELECT pt + ,exposure_uv + FROM test1 + ) m0 + LEFT JOIN + ( + SELECT pt + ,exposure_uv + FROM test1 + ) m1 + ON m0.pt = m1.pt + LEFT JOIN + ( + SELECT pt + ,exposure_uv + FROM test1 + ) m2 + ON m0.pt = m2.pt +) c0 +ORDER BY exposure_uv +settings join_use_nulls = 1; + +SELECT + pt AS pt, + exposure_uv AS exposure_uv +FROM +( + SELECT + pt + FROM test1 +) AS m0 +FULL OUTER JOIN +( + SELECT + pt, + exposure_uv + FROM test1 +) AS m1 ON m0.pt = m1.pt; From f316914aed71f2e90caf0a6af707cb5756bdbec8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 3 Jul 2023 15:29:57 +0000 Subject: [PATCH 02/22] Fix another one key. --- .../Optimizations/liftUpFunctions.cpp | 10 +- ...nd_columns_with_same_names_bug_2.reference | 3 + ...ting_and_columns_with_same_names_bug_2.sql | 107 ++++++++++++++++++ 3 files changed, 114 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug_2.reference create mode 100644 tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug_2.sql diff --git a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp index b2c3f3b4a6d..47b4e31ed32 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp @@ -29,16 +29,14 @@ const DB::DataStream & getChildOutputStream(DB::QueryPlan::Node & node) namespace DB::QueryPlanOptimizations { -/// This is a check that output columns with the same name have the same types. -/// This is ok to have such a situation in DAG, but not for Block. -/// TODO: we should have a different data structure for headers. +/// This is a check that output columns does not have the same name +/// This is ok for DAG, but may introduce a bug in a SotringStep cause columns are selected by name. static bool areOutputsAreConvertableToBlock(const ActionsDAG::NodeRawConstPtrs & outputs) { - std::unordered_map name_to_type; + std::unordered_set names; for (const auto & output : outputs) { - auto [it, inserted] = name_to_type.emplace(output->result_name, output->result_type.get()); - if (!inserted && !it->second->equals(*output->result_type)) + if (!names.emplace(output->result_name).second) return false; } diff --git a/tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug_2.reference b/tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug_2.reference new file mode 100644 index 00000000000..bcc55e50958 --- /dev/null +++ b/tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug_2.reference @@ -0,0 +1,3 @@ +20230626 0.3156979034107179 \N \N +20230626 0.2624629016490004 \N \N +20230626 0.19390556368960468 \N \N diff --git a/tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug_2.sql b/tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug_2.sql new file mode 100644 index 00000000000..b0221635fe9 --- /dev/null +++ b/tests/queries/0_stateless/02789_functions_after_sorting_and_columns_with_same_names_bug_2.sql @@ -0,0 +1,107 @@ +create table test1 ( + `pt` String, + `brand_name` String, + `total_indirect_order_cnt` Float64, + `total_indirect_gmv` Float64 +) ENGINE = Memory; + +create table test2 ( + `pt` String, + `brand_name` String, + `exposure_uv` Float64, + `click_uv` Float64 +) ENGINE = Memory; + +INSERT INTO test1 (`pt`, `brand_name`, `total_indirect_order_cnt`, `total_indirect_gmv`) VALUES ('20230625', 'LINING', 2232, 1008710), ('20230625', 'adidas', 125, 58820), ('20230625', 'Nike', 1291, 1033020), ('20230626', 'Nike', 1145, 938926), ('20230626', 'LINING', 1904, 853336), ('20230626', 'adidas', 133, 62546), ('20220626', 'LINING', 3747, 1855203), ('20220626', 'Nike', 2295, 1742665), ('20220626', 'adidas', 302, 122388); + +INSERT INTO test2 (`pt`, `brand_name`, `exposure_uv`, `click_uv`) VALUES ('20230625', 'Nike', 2012913, 612831), ('20230625', 'adidas', 480277, 96176), ('20230625', 'LINING', 2474234, 627814), ('20230626', 'Nike', 1934666, 610770), ('20230626', 'adidas', 469904, 91117), ('20230626', 'LINING', 2285142, 599765), ('20220626', 'Nike', 2979656, 937166), ('20220626', 'adidas', 704751, 124250), ('20220626', 'LINING', 3163884, 1010221); + +SELECT * FROM ( + SELECT m0.pt AS pt + ,m0.`uvctr` AS uvctr + ,round(m1.uvctr,4) AS uvctr_hb_last_value + ,round(m2.uvctr,4) AS uvctr_tb_last_value + FROM + ( + SELECT m0.pt AS pt + ,COALESCE(m0.brand_name,m1.brand_name) AS brand_name + ,if(isNaN(`click_uv` / `exposure_uv`) OR isInfinite(`click_uv` / `exposure_uv`),NULL,`click_uv` / `exposure_uv`) AS `uvctr` + FROM + ( + SELECT pt AS pt + ,brand_name AS `brand_name` + ,exposure_uv AS `exposure_uv` + ,click_uv AS `click_uv` + FROM test2 + WHERE pt = '20230626' + ) m0 + FULL JOIN + ( + SELECT pt AS pt + ,brand_name AS `brand_name` + ,total_indirect_order_cnt AS `total_indirect_order_cnt` + ,total_indirect_gmv AS `total_indirect_gmv` + FROM test1 + WHERE pt = '20230626' + ) m1 + ON m0.brand_name = m1.brand_name AND m0.pt = m1.pt + ) m0 + LEFT JOIN + ( + SELECT m0.pt AS pt + ,if(isNaN(`click_uv` / `exposure_uv`) OR isInfinite(`click_uv` / `exposure_uv`),NULL,`click_uv` / `exposure_uv`) AS `uvctr` + ,COALESCE(m0.brand_name,m1.brand_name) AS brand_name + ,`exposure_uv` AS `exposure_uv` + ,`click_uv` + FROM + ( + SELECT pt AS pt + ,brand_name AS `brand_name` + ,exposure_uv AS `exposure_uv` + ,click_uv AS `click_uv` + FROM test2 + WHERE pt = '20230625' + ) m0 + FULL JOIN + ( + SELECT pt AS pt + ,brand_name AS `brand_name` + ,total_indirect_order_cnt AS `total_indirect_order_cnt` + ,total_indirect_gmv AS `total_indirect_gmv` + FROM test1 + WHERE pt = '20230625' + ) m1 + ON m0.brand_name = m1.brand_name AND m0.pt = m1.pt + ) m1 + ON m0.brand_name = m1.brand_name AND m0.pt = m1.pt + LEFT JOIN + ( + SELECT m0.pt AS pt + ,if(isNaN(`click_uv` / `exposure_uv`) OR isInfinite(`click_uv` / `exposure_uv`),NULL,`click_uv` / `exposure_uv`) AS `uvctr` + ,COALESCE(m0.brand_name,m1.brand_name) AS brand_name + ,`exposure_uv` AS `exposure_uv` + ,`click_uv` + FROM + ( + SELECT pt AS pt + ,brand_name AS `brand_name` + ,exposure_uv AS `exposure_uv` + ,click_uv AS `click_uv` + FROM test2 + WHERE pt = '20220626' + ) m0 + FULL JOIN + ( + SELECT pt AS pt + ,brand_name AS `brand_name` + ,total_indirect_order_cnt AS `total_indirect_order_cnt` + ,total_indirect_gmv AS `total_indirect_gmv` + FROM test1 + WHERE pt = '20220626' + ) m1 + ON m0.brand_name = m1.brand_name AND m0.pt = m1.pt + ) m2 + ON m0.brand_name = m2.brand_name AND m0.pt = m2.pt +) c0 +ORDER BY pt ASC, uvctr DESC; + From d891c4c9a218784d1717710a676b3a132339b14b Mon Sep 17 00:00:00 2001 From: Lars Eidnes Date: Wed, 12 Jul 2023 15:06:01 +0200 Subject: [PATCH 03/22] MaterializedMySQL: Delay disconnection of mysqlxx::Pool::Entry until refcount==0 If Entry::disconnect() is called, it is removed from its pool and underlying mysqlxx::Connection is disconnected. Since Entry objects are copyable and some of them could be alive when disconnect() is called which would lead to undefined behavior. Suggesting to disconnect it when the last object is being destroyed. --- src/Common/mysqlxx/Pool.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Common/mysqlxx/Pool.cpp b/src/Common/mysqlxx/Pool.cpp index d10889d1f97..64a69c48e1d 100644 --- a/src/Common/mysqlxx/Pool.cpp +++ b/src/Common/mysqlxx/Pool.cpp @@ -25,8 +25,6 @@ void Pool::Entry::incrementRefCount() /// First reference, initialize thread if (data->ref_count.fetch_add(1) == 0) mysql_thread_init(); - - chassert(!data->removed_from_pool); } @@ -43,7 +41,10 @@ void Pool::Entry::decrementRefCount() /// In Pool::Entry::disconnect() we remove connection from the list of pool's connections. /// So now we must deallocate the memory. if (data->removed_from_pool) + { + data->conn.disconnect(); ::delete data; + } } } @@ -230,8 +231,6 @@ void Pool::removeConnection(Connection* connection) std::lock_guard lock(mutex); if (connection) { - if (!connection->removed_from_pool) - connection->conn.disconnect(); connections.remove(connection); connection->removed_from_pool = true; } @@ -240,6 +239,7 @@ void Pool::removeConnection(Connection* connection) void Pool::Entry::disconnect() { + // Remove the Entry from the Pool. Actual disconnection is delayed until refcount == 0. pool->removeConnection(data); } From 254b0a65b3c82124d4053e6d4d6099ec23c8980e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 14 Jul 2023 20:03:48 +0200 Subject: [PATCH 04/22] Update CCTZ --- contrib/cctz | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/cctz b/contrib/cctz index 5e05432420f..8529bcef5cd 160000 --- a/contrib/cctz +++ b/contrib/cctz @@ -1 +1 @@ -Subproject commit 5e05432420f9692418e2e12aff09859e420b14a2 +Subproject commit 8529bcef5cd996b7c0f4d7475286b76b5d126c4c From a4f7b7717768893983be2973cef9b1a5684b1949 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 09:21:00 +0000 Subject: [PATCH 05/22] Add array_concat_agg for compatibility with BigQuery --- .../AggregateFunctionFactory.cpp | 29 ++++++++++++++++++- .../AggregateFunctionFactory.h | 3 ++ .../registerAggregateFunctions.cpp | 8 +++++ src/Common/IFactoryWithAliases.h | 20 ++++++------- .../02813_array_concat_agg.reference | 5 ++++ .../0_stateless/02813_array_concat_agg.sql | 9 ++++++ 6 files changed, 63 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02813_array_concat_agg.reference create mode 100644 tests/queries/0_stateless/02813_array_concat_agg.sql diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index 6cacf66500f..f2dbb931b1f 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -59,6 +59,34 @@ void AggregateFunctionFactory::registerFunction(const String & name, Value creat } } +void AggregateFunctionFactory::registerAliasForAggregateFunctionWithCombinator( + const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness) +{ + if (!isAggregateFunctionName(real_name)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "{}: can't create alias '{}', the real name '{}' is not registered", + getFactoryName(), + alias_name, + real_name); + + auto alias_name_lowercase = Poco::toLower(alias_name); + + if (aggregate_functions.contains(alias_name) || case_insensitive_aggregate_functions.contains(alias_name_lowercase)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "{}: the alias name '{}' is already registered as real name", getFactoryName(), alias_name); + + if (case_sensitiveness == CaseInsensitive) + { + if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_name).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", getFactoryName(), alias_name); + case_insensitive_name_mapping[alias_name_lowercase] = real_name; + } + + if (!aliases.emplace(alias_name, real_name).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: alias name '{}' is not unique", getFactoryName(), alias_name); +} + static DataTypes convertLowCardinalityTypesToNested(const DataTypes & types) { DataTypes res_types; @@ -222,7 +250,6 @@ AggregateFunctionPtr AggregateFunctionFactory::tryGet( : nullptr; } - std::optional AggregateFunctionFactory::tryGetProperties(String name) const { if (name.size() > MAX_AGGREGATE_FUNCTION_NAME_LENGTH) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.h b/src/AggregateFunctions/AggregateFunctionFactory.h index dab0d28e851..6c2b539bd8c 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.h +++ b/src/AggregateFunctions/AggregateFunctionFactory.h @@ -62,6 +62,9 @@ public: Value creator, CaseSensitiveness case_sensitiveness = CaseSensitive); + void registerAliasForAggregateFunctionWithCombinator( + const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive); + /// Throws an exception if not found. AggregateFunctionPtr get(const String & name, diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 91248a52ae9..87d96f0d1ce 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -193,6 +193,14 @@ void registerAggregateFunctions() registerAggregateFunctionCombinatorDistinct(factory); registerAggregateFunctionCombinatorMap(factory); } + + { + auto & factory = AggregateFunctionFactory::instance(); + + /// Must register after registerCombinator + factory.registerAliasForAggregateFunctionWithCombinator( + "array_concat_agg", "groupArrayArray", AggregateFunctionFactory::CaseInsensitive); + } } } diff --git a/src/Common/IFactoryWithAliases.h b/src/Common/IFactoryWithAliases.h index b2ac4ab289e..af5656ffb75 100644 --- a/src/Common/IFactoryWithAliases.h +++ b/src/Common/IFactoryWithAliases.h @@ -10,6 +10,8 @@ namespace DB { +class AggregateFunctionFactory; + namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -22,6 +24,7 @@ namespace ErrorCodes template class IFactoryWithAliases : public IHints<2, IFactoryWithAliases> { + friend AggregateFunctionFactory; protected: using Value = ValueType; @@ -55,9 +58,9 @@ public: const String factory_name = getFactoryName(); String real_dict_name; - if (creator_map.count(real_name)) + if (creator_map.contains(real_name)) real_dict_name = real_name; - else if (auto real_name_lowercase = Poco::toLower(real_name); case_insensitive_creator_map.count(real_name_lowercase)) + else if (auto real_name_lowercase = Poco::toLower(real_name); case_insensitive_creator_map.contains(real_name_lowercase)) real_dict_name = real_name_lowercase; else throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: can't create alias '{}', the real name '{}' is not registered", @@ -65,7 +68,7 @@ public: String alias_name_lowercase = Poco::toLower(alias_name); - if (creator_map.count(alias_name) || case_insensitive_creator_map.count(alias_name_lowercase)) + if (creator_map.contains(alias_name) || case_insensitive_creator_map.contains(alias_name_lowercase)) throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: the alias name '{}' is already registered as real name", factory_name, alias_name); @@ -93,7 +96,7 @@ public: bool isCaseInsensitive(const String & name) const { String name_lowercase = Poco::toLower(name); - return getCaseInsensitiveMap().count(name_lowercase) || case_insensitive_aliases.count(name_lowercase); + return getCaseInsensitiveMap().contains(name_lowercase) || case_insensitive_aliases.contains(name_lowercase); } const String & aliasTo(const String & name) const @@ -106,14 +109,11 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: name '{}' is not alias", getFactoryName(), name); } - bool isAlias(const String & name) const - { - return aliases.count(name) || case_insensitive_aliases.contains(name); - } + bool isAlias(const String & name) const { return aliases.contains(name) || case_insensitive_aliases.contains(name); } bool hasNameOrAlias(const String & name) const { - return getMap().count(name) || getCaseInsensitiveMap().count(name) || isAlias(name); + return getMap().contains(name) || getCaseInsensitiveMap().contains(name) || isAlias(name); } /// Return the canonical name (the name used in registration) if it's different from `name`. @@ -129,7 +129,7 @@ public: private: using InnerMap = std::unordered_map; // name -> creator - using AliasMap = std::unordered_map; // alias -> original type + using AliasMap = std::unordered_map; // alias -> original name virtual const InnerMap & getMap() const = 0; virtual const InnerMap & getCaseInsensitiveMap() const = 0; diff --git a/tests/queries/0_stateless/02813_array_concat_agg.reference b/tests/queries/0_stateless/02813_array_concat_agg.reference new file mode 100644 index 00000000000..7144a499922 --- /dev/null +++ b/tests/queries/0_stateless/02813_array_concat_agg.reference @@ -0,0 +1,5 @@ +[1,2,3,4,5,6] +[1,2,3,4,5,6] +1 [1,2,3] +2 [4,5] +3 [6] diff --git a/tests/queries/0_stateless/02813_array_concat_agg.sql b/tests/queries/0_stateless/02813_array_concat_agg.sql new file mode 100644 index 00000000000..94fe133db7d --- /dev/null +++ b/tests/queries/0_stateless/02813_array_concat_agg.sql @@ -0,0 +1,9 @@ +drop table if exists t; + +create table t (n UInt32, a Array(Int32)) engine=Memory; +insert into t values (1, [1,2,3]), (2, [4,5]), (3, [6]); + +select array_concat_agg(a) from t; +select ArrAy_cOncAt_aGg(a) from t; +select n, array_concat_agg(a) from t group by n order by n; +drop table t; From 4955e07326c4220af989a8938dda9fc3c783c9de Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 09:27:34 +0000 Subject: [PATCH 06/22] add docs --- .../reference/arrayconcatagg.md | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md b/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md new file mode 100644 index 00000000000..50e4ed63787 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md @@ -0,0 +1,32 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/arrayconcatagg +sidebar_position: 110 +--- + +# array_concat_agg +- Alias of `groupArrayArray`. The function is case insensitive. + +**Exampla** + +```text +SELECT * +FROM t + +┌─a───────┐ +│ [1,2,3] │ +│ [4,5] │ +│ [6] │ +└─────────┘ + +``` + +Query: + +```sql +SELECT array_concat_agg(a) AS a +FROM t + +┌─a─────────────┐ +│ [1,2,3,4,5,6] │ +└───────────────┘ +``` From f59370e5f98f72fa136529ed4a4f545faf18b510 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 09:50:29 +0000 Subject: [PATCH 07/22] Fix style --- .../reference/{arrayconcatagg.md => array_concat_agg.md} | 2 +- src/AggregateFunctions/registerAggregateFunctions.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) rename docs/en/sql-reference/aggregate-functions/reference/{arrayconcatagg.md => array_concat_agg.md} (97%) diff --git a/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md b/docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md similarity index 97% rename from docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md rename to docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md index 50e4ed63787..db1f1a10859 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md +++ b/docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md @@ -6,7 +6,7 @@ sidebar_position: 110 # array_concat_agg - Alias of `groupArrayArray`. The function is case insensitive. -**Exampla** +**Example** ```text SELECT * diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 87d96f0d1ce..92484ad93ff 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -194,7 +194,7 @@ void registerAggregateFunctions() registerAggregateFunctionCombinatorMap(factory); } - { + { auto & factory = AggregateFunctionFactory::instance(); /// Must register after registerCombinator From 6899070f95ea81b6a77090f06fb45332f475d349 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 09:54:13 +0000 Subject: [PATCH 08/22] fix --- .../reference/{array_concat_agg.md => arrayconcatagg.md} | 2 +- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) rename docs/en/sql-reference/aggregate-functions/reference/{array_concat_agg.md => arrayconcatagg.md} (86%) diff --git a/docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md b/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md similarity index 86% rename from docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md rename to docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md index db1f1a10859..3c71129bdb5 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/array_concat_agg.md +++ b/docs/en/sql-reference/aggregate-functions/reference/arrayconcatagg.md @@ -1,5 +1,5 @@ --- -slug: /en/sql-reference/aggregate-functions/reference/arrayconcatagg +slug: /en/sql-reference/aggregate-functions/reference/array_concat_agg sidebar_position: 110 --- diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 2802e52c288..fc2cd5640b1 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -990,6 +990,7 @@ addressToLine addressToLineWithInlines addressToSymbol adviced +agg aggregatefunction aggregatingmergetree aggregatio From bbade814f3e59bf31fb86cadb17b0ffed1661257 Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 16 Jul 2023 17:10:28 +0000 Subject: [PATCH 09/22] fix --- .../AggregateFunctionFactory.cpp | 28 ------------- .../AggregateFunctionFactory.h | 3 -- .../AggregateFunctionGroupArray.cpp | 1 + .../registerAggregateFunctions.cpp | 8 ---- src/Common/IFactoryWithAliases.h | 40 +++++++++---------- 5 files changed, 21 insertions(+), 59 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index f2dbb931b1f..f52c9ac8510 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -59,34 +59,6 @@ void AggregateFunctionFactory::registerFunction(const String & name, Value creat } } -void AggregateFunctionFactory::registerAliasForAggregateFunctionWithCombinator( - const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness) -{ - if (!isAggregateFunctionName(real_name)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "{}: can't create alias '{}', the real name '{}' is not registered", - getFactoryName(), - alias_name, - real_name); - - auto alias_name_lowercase = Poco::toLower(alias_name); - - if (aggregate_functions.contains(alias_name) || case_insensitive_aggregate_functions.contains(alias_name_lowercase)) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "{}: the alias name '{}' is already registered as real name", getFactoryName(), alias_name); - - if (case_sensitiveness == CaseInsensitive) - { - if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_name).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", getFactoryName(), alias_name); - case_insensitive_name_mapping[alias_name_lowercase] = real_name; - } - - if (!aliases.emplace(alias_name, real_name).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: alias name '{}' is not unique", getFactoryName(), alias_name); -} - static DataTypes convertLowCardinalityTypesToNested(const DataTypes & types) { DataTypes res_types; diff --git a/src/AggregateFunctions/AggregateFunctionFactory.h b/src/AggregateFunctions/AggregateFunctionFactory.h index 6c2b539bd8c..dab0d28e851 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.h +++ b/src/AggregateFunctions/AggregateFunctionFactory.h @@ -62,9 +62,6 @@ public: Value creator, CaseSensitiveness case_sensitiveness = CaseSensitive); - void registerAliasForAggregateFunctionWithCombinator( - const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive); - /// Throws an exception if not found. AggregateFunctionPtr get(const String & name, diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp index bb1368b9ff8..a09e9ddfbf6 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.cpp @@ -125,6 +125,7 @@ void registerAggregateFunctionGroupArray(AggregateFunctionFactory & factory) AggregateFunctionProperties properties = { .returns_default_when_only_null = false, .is_order_dependent = true }; factory.registerFunction("groupArray", { createAggregateFunctionGroupArray, properties }); + factory.registerAliasUnchecked("array_concat_agg", "groupArrayArray", AggregateFunctionFactory::CaseInsensitive); factory.registerFunction("groupArraySample", { createAggregateFunctionGroupArraySample, properties }); factory.registerFunction("groupArrayLast", { createAggregateFunctionGroupArray, properties }); } diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 92484ad93ff..91248a52ae9 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -193,14 +193,6 @@ void registerAggregateFunctions() registerAggregateFunctionCombinatorDistinct(factory); registerAggregateFunctionCombinatorMap(factory); } - - { - auto & factory = AggregateFunctionFactory::instance(); - - /// Must register after registerCombinator - factory.registerAliasForAggregateFunctionWithCombinator( - "array_concat_agg", "groupArrayArray", AggregateFunctionFactory::CaseInsensitive); - } } } diff --git a/src/Common/IFactoryWithAliases.h b/src/Common/IFactoryWithAliases.h index af5656ffb75..07440dd2463 100644 --- a/src/Common/IFactoryWithAliases.h +++ b/src/Common/IFactoryWithAliases.h @@ -10,8 +10,6 @@ namespace DB { -class AggregateFunctionFactory; - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -24,7 +22,6 @@ namespace ErrorCodes template class IFactoryWithAliases : public IHints<2, IFactoryWithAliases> { - friend AggregateFunctionFactory; protected: using Value = ValueType; @@ -55,35 +52,38 @@ public: { const auto & creator_map = getMap(); const auto & case_insensitive_creator_map = getCaseInsensitiveMap(); - const String factory_name = getFactoryName(); - String real_dict_name; - if (creator_map.contains(real_name)) - real_dict_name = real_name; - else if (auto real_name_lowercase = Poco::toLower(real_name); case_insensitive_creator_map.contains(real_name_lowercase)) - real_dict_name = real_name_lowercase; - else - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: can't create alias '{}', the real name '{}' is not registered", - factory_name, alias_name, real_name); + auto real_name_lowercase = Poco::toLower(real_name); + if (!creator_map.contains(real_name) && !case_insensitive_creator_map.contains(real_name_lowercase)) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "{}: can't create alias '{}', the real name '{}' is not registered", + getFactoryName(), + alias_name, + real_name); + registerAliasUnchecked(alias_name, real_name, case_sensitiveness); + } + + /// We need sure the real_name exactly exists when call the function directly. + void registerAliasUnchecked(const String & alias_name, const String & real_name, CaseSensitiveness case_sensitiveness = CaseSensitive) + { String alias_name_lowercase = Poco::toLower(alias_name); - - if (creator_map.contains(alias_name) || case_insensitive_creator_map.contains(alias_name_lowercase)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: the alias name '{}' is already registered as real name", - factory_name, alias_name); + String real_name_lowercase = Poco::toLower(real_name); + const String factory_name = getFactoryName(); if (case_sensitiveness == CaseInsensitive) { - if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_dict_name).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", - factory_name, alias_name); + if (!case_insensitive_aliases.emplace(alias_name_lowercase, real_name).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: case insensitive alias name '{}' is not unique", factory_name, alias_name); case_insensitive_name_mapping[alias_name_lowercase] = real_name; } - if (!aliases.emplace(alias_name, real_dict_name).second) + if (!aliases.emplace(alias_name, real_name).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "{}: alias name '{}' is not unique", factory_name, alias_name); } + std::vector getAllRegisteredNames() const override { std::vector result; From 4e727cdb2a870c6d0876ba0f251ccc161e0c9777 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 17 Jul 2023 03:21:24 +0300 Subject: [PATCH 10/22] Update liftUpFunctions.cpp --- src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp index 47b4e31ed32..34a1fc2bb88 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpFunctions.cpp @@ -31,7 +31,7 @@ namespace DB::QueryPlanOptimizations /// This is a check that output columns does not have the same name /// This is ok for DAG, but may introduce a bug in a SotringStep cause columns are selected by name. -static bool areOutputsAreConvertableToBlock(const ActionsDAG::NodeRawConstPtrs & outputs) +static bool areOutputsConvertableToBlock(const ActionsDAG::NodeRawConstPtrs & outputs) { std::unordered_set names; for (const auto & output : outputs) @@ -72,7 +72,7 @@ size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan: if (unneeded_for_sorting->trivial()) return 0; - if (!areOutputsAreConvertableToBlock(needed_for_sorting->getOutputs())) + if (!areOutputsConvertableToBlock(needed_for_sorting->getOutputs())) return 0; // Sorting (parent_node) -> Expression (child_node) From c374653c7bfc7450590d6a787bd4f413d15a76af Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 17 Jul 2023 12:01:47 +0200 Subject: [PATCH 11/22] Revert "Merge pull request #52138 from azat/decompressor-inode" This reverts commit 6524031348c3a1148e27ef926baabb35e833a09c, reversing changes made to 9bf114f9a36556d6f227dea0fa91be131ee99710. This was not a good idea, since the underlying problem was that `/proc/self/exe` was pointing to `qemu-$ARCH-static` (because the code uses realpath() over normal interface readlink(), which is not caught by the qemu linux-user). And this means that later, it will try to overwrite incorrect binary and then execute some garbage. Signed-off-by: Azat Khuzhin --- .../decompressor.cpp | 93 +++++++++---------- 1 file changed, 46 insertions(+), 47 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 4a4985120fd..d41b9b1ebe1 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -430,58 +430,55 @@ int main(int/* argc*/, char* argv[]) return 1; } - int lock = -1; - /// Protection from double decompression #if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// get inode of this executable uint64_t inode = getInode(self); - /// In some cases /proc/self/maps may not contain the inode for the - /// /proc/self/exe, one of such examples are using qemu-*-static, in this - /// case maps will be proxied through the qemu, and it will remove - /// information about itself from it. - if (inode != 0) + if (inode == 0) { - std::stringstream lock_path; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; - lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); - if (lock < 0) + std::cerr << "Unable to obtain inode." << std::endl; + return 1; + } + + std::stringstream lock_path; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + lock_path << "/tmp/" << name << ".decompression." << inode << ".lock"; + int lock = open(lock_path.str().c_str(), O_CREAT | O_RDWR, 0666); + if (lock < 0) + { + perror("lock open"); + return 1; + } + + /// lock file should be closed on exec call + fcntl(lock, F_SETFD, FD_CLOEXEC); + + if (lockf(lock, F_LOCK, 0)) + { + perror("lockf"); + return 1; + } + + /// inconsistency in WSL1 Ubuntu - inode reported in /proc/self/maps is a 64bit to + /// 32bit conversion of input_info.st_ino + if (input_info.st_ino & 0xFFFFFFFF00000000 && !(inode & 0xFFFFFFFF00000000)) + input_info.st_ino &= 0x00000000FFFFFFFF; + + /// if decompression was performed by another process since this copy was started + /// then file referred by path "self" is already pointing to different inode + if (input_info.st_ino != inode) + { + struct stat lock_info; + if (0 != fstat(lock, &lock_info)) { - perror("lock open"); + perror("fstat lock"); return 1; } - /// lock file should be closed on exec call - fcntl(lock, F_SETFD, FD_CLOEXEC); + /// size 1 of lock file indicates that another decompressor has found active executable + if (lock_info.st_size == 1) + execv(self, argv); - if (lockf(lock, F_LOCK, 0)) - { - perror("lockf"); - return 1; - } - - /// inconsistency in WSL1 Ubuntu - inode reported in /proc/self/maps is a 64bit to - /// 32bit conversion of input_info.st_ino - if (input_info.st_ino & 0xFFFFFFFF00000000 && !(inode & 0xFFFFFFFF00000000)) - input_info.st_ino &= 0x00000000FFFFFFFF; - - /// if decompression was performed by another process since this copy was started - /// then file referred by path "self" is already pointing to different inode - if (input_info.st_ino != inode) - { - struct stat lock_info; - if (0 != fstat(lock, &lock_info)) - { - perror("fstat lock"); - return 1; - } - - /// size 1 of lock file indicates that another decompressor has found active executable - if (lock_info.st_size == 1) - execv(self, argv); - - printf("No target executable - decompression only was performed.\n"); - return 0; - } + printf("No target executable - decompression only was performed.\n"); + return 0; } #endif @@ -549,19 +546,21 @@ int main(int/* argc*/, char* argv[]) if (has_exec) { +#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// write one byte to the lock in case other copies of compressed are running to indicate that /// execution should be performed - if (lock >= 0) - write(lock, "1", 1); + write(lock, "1", 1); +#endif execv(self, argv); /// This part of code will be reached only if error happened perror("execv"); return 1; } +#if !defined(OS_DARWIN) && !defined(OS_FREEBSD) /// since inodes can be reused - it's a precaution if lock file already exists and have size of 1 - if (lock >= 0) - ftruncate(lock, 0); + ftruncate(lock, 0); +#endif printf("No target executable - decompression only was performed.\n"); } From 16165d9498cbebd3ecd02df87480230fd0ed880e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 17 Jul 2023 12:02:54 +0200 Subject: [PATCH 12/22] Improve error messages for decompressor Signed-off-by: Azat Khuzhin --- utils/self-extracting-executable/decompressor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index d41b9b1ebe1..567d9088f13 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -435,7 +435,7 @@ int main(int/* argc*/, char* argv[]) uint64_t inode = getInode(self); if (inode == 0) { - std::cerr << "Unable to obtain inode." << std::endl; + std::cerr << "Unable to obtain inode for exe '" << self << "'." << std::endl; return 1; } From 1fb7605fb4225225d492cf63f7f048d594e89fc3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 17 Jul 2023 12:09:11 +0200 Subject: [PATCH 13/22] Fix self extracting binaries under qemu linux-user (qemu-$ARCH-static) The problem was that the decompressor uses realpath(/proc/self/exe) instead of readlink(/proc/self/exe), while realpath() does lots of trickerly [1] which leads to bypassing qemu linux-user override [2] of /proc/self/exe to the executable with with it had been called -- and the reason for this is that the getpid() after unshare returns 1, while reading /proc/self returns the pid that was before unshare (from the chroot) [3]. [1]: https://github.com/bminor/glibc/blob/4290aed05135ae4c0272006442d147f2155e70d7/stdlib/canonicalize.c#L223 [2]: https://github.com/qemu/qemu/blob/ed8ad9728a9c0eec34db9dff61dfa2f1dd625637/linux-user/syscall.c#L8634 [3]: https://gist.github.com/azat/fcbd8b6c26afd505ae5f3387fc15f0e2 But note, that even after this patch qemu without binfmt will not work, due to internally the code calls execv() while qemu does not handle it (see [4]). [4]: https://patchwork.kernel.org/project/qemu-devel/patch/1453091602-21843-1-git-send-email-petrosagg@gmail.com/ Signed-off-by: Azat Khuzhin --- utils/self-extracting-executable/decompressor.cpp | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 567d9088f13..91f4bea5a5b 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -362,11 +362,12 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress #else - int read_exe_path(char *exe, size_t/* buf_sz*/) + int read_exe_path(char *exe, size_t buf_sz) { - if (realpath("/proc/self/exe", exe) == nullptr) - return 1; - return 0; + ssize_t n = readlink("/proc/self/exe", exe, buf_sz - 1); + if (n > 0) + exe[n] = '\0'; + return n > 0 && n < static_cast(buf_sz); } #endif From 10c15a6548a27bbb51eb04082a11fd110a4642a5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 10:33:09 +0000 Subject: [PATCH 14/22] Refactor limit_jobs.cmake --- cmake/limit_jobs.cmake | 53 ++++++++++++++++++++---------------------- 1 file changed, 25 insertions(+), 28 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 3a33b3b9989..b2a54fb8d1a 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -1,16 +1,14 @@ -# Usage: -# set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") # In megabytes -# set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") +# Usage from CMake: +# set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") # megabyte +# set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") # megabyte # include (cmake/limit_jobs.cmake) cmake_host_system_information(RESULT TOTAL_PHYSICAL_MEMORY QUERY TOTAL_PHYSICAL_MEMORY) # Not available under freebsd cmake_host_system_information(RESULT NUMBER_OF_LOGICAL_CORES QUERY NUMBER_OF_LOGICAL_CORES) -# 1 if not set -option(PARALLEL_COMPILE_JOBS "Maximum number of concurrent compilation jobs" "") - -# 1 if not set -option(PARALLEL_LINK_JOBS "Maximum number of concurrent link jobs" "") +# Set to disable the automatic job-limiting +option(PARALLEL_COMPILE_JOBS "Maximum number of concurrent compilation jobs" OFF) +option(PARALLEL_LINK_JOBS "Maximum number of concurrent link jobs" OFF) if (NOT PARALLEL_COMPILE_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY) math(EXPR PARALLEL_COMPILE_JOBS ${TOTAL_PHYSICAL_MEMORY}/${MAX_COMPILER_MEMORY}) @@ -19,17 +17,10 @@ if (NOT PARALLEL_COMPILE_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY) set (PARALLEL_COMPILE_JOBS 1) endif () if (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) - set (PARALLEL_COMPILE_JOBS_LESS TRUE) + set (PARALLEL_COMPILE_JOBS_UNDERUTILIZING TRUE) endif() endif () -if (PARALLEL_COMPILE_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) - set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) - string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) - set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS}) -endif () - - if (NOT PARALLEL_LINK_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_LINKER_MEMORY) math(EXPR PARALLEL_LINK_JOBS ${TOTAL_PHYSICAL_MEMORY}/${MAX_LINKER_MEMORY}) @@ -37,7 +28,7 @@ if (NOT PARALLEL_LINK_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_LINKER_MEMORY) set (PARALLEL_LINK_JOBS 1) endif () if (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) - set (PARALLEL_LINK_JOBS_LESS TRUE) + set (PARALLEL_LINK_JOBS_UNDERUTILIZING TRUE) endif() endif () @@ -52,20 +43,26 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO AND PARALLE set (PARALLEL_LINK_JOBS 2) endif() +message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_COMPILE_JOBS} linker jobs.") + +if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) + if (PARALLEL_COMPILE_JOBS_UNDERUTILIZING) + message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") + endif() + if (PARALLEL_LINK_JOBS_UNDERUTILIZING) + message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") + endif() +endif () + +if (PARALLEL_COMPILE_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) + set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) + string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) + set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS}) +endif () + if (PARALLEL_LINK_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES)) set(CMAKE_JOB_POOL_LINK link_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_LINK ${CMAKE_JOB_POOL_LINK}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_LINK}=${PARALLEL_LINK_JOBS}) endif () -if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) - message(STATUS - "${CMAKE_CURRENT_SOURCE_DIR}: Have ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. - Limiting concurrent linkers jobs to ${PARALLEL_LINK_JOBS} and compiler jobs to ${PARALLEL_COMPILE_JOBS} (system has ${NUMBER_OF_LOGICAL_CORES} logical cores)") - if (PARALLEL_COMPILE_JOBS_LESS) - message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") - endif() - if (PARALLEL_LINK_JOBS_LESS) - message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") - endif() -endif () From 7204f7054788302e37be3e79ddd3a0ff87b93f01 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 11:52:00 +0000 Subject: [PATCH 15/22] cmake_host_system_information provides TOTAL_PHYSICAL_MEMORY on all platforms, including FreeBSD --- cmake/limit_jobs.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index b2a54fb8d1a..2ca9e5519d7 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -3,14 +3,14 @@ # set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") # megabyte # include (cmake/limit_jobs.cmake) -cmake_host_system_information(RESULT TOTAL_PHYSICAL_MEMORY QUERY TOTAL_PHYSICAL_MEMORY) # Not available under freebsd +cmake_host_system_information(RESULT TOTAL_PHYSICAL_MEMORY QUERY TOTAL_PHYSICAL_MEMORY) cmake_host_system_information(RESULT NUMBER_OF_LOGICAL_CORES QUERY NUMBER_OF_LOGICAL_CORES) # Set to disable the automatic job-limiting option(PARALLEL_COMPILE_JOBS "Maximum number of concurrent compilation jobs" OFF) option(PARALLEL_LINK_JOBS "Maximum number of concurrent link jobs" OFF) -if (NOT PARALLEL_COMPILE_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY) +if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) math(EXPR PARALLEL_COMPILE_JOBS ${TOTAL_PHYSICAL_MEMORY}/${MAX_COMPILER_MEMORY}) if (NOT PARALLEL_COMPILE_JOBS) @@ -21,7 +21,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_COMPILER_MEMORY) endif() endif () -if (NOT PARALLEL_LINK_JOBS AND TOTAL_PHYSICAL_MEMORY AND MAX_LINKER_MEMORY) +if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) math(EXPR PARALLEL_LINK_JOBS ${TOTAL_PHYSICAL_MEMORY}/${MAX_LINKER_MEMORY}) if (NOT PARALLEL_LINK_JOBS) From 8f1e56167c430783841abca00dd16c217cfcec13 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 11:53:12 +0000 Subject: [PATCH 16/22] NUMBER_OF_TOTAL_CORES is on all platforms != 0 --- cmake/limit_jobs.cmake | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 2ca9e5519d7..b145c069e37 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -16,7 +16,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) if (NOT PARALLEL_COMPILE_JOBS) set (PARALLEL_COMPILE_JOBS 1) endif () - if (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) + if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) set (PARALLEL_COMPILE_JOBS_UNDERUTILIZING TRUE) endif() endif () @@ -27,7 +27,7 @@ if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) if (NOT PARALLEL_LINK_JOBS) set (PARALLEL_LINK_JOBS 1) endif () - if (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) + if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) set (PARALLEL_LINK_JOBS_UNDERUTILIZING TRUE) endif() endif () @@ -54,13 +54,13 @@ if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) endif() endif () -if (PARALLEL_COMPILE_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) +if (PARALLEL_COMPILE_JOBS AND (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS}) endif () -if (PARALLEL_LINK_JOBS AND (NOT NUMBER_OF_LOGICAL_CORES OR PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES)) +if (PARALLEL_LINK_JOBS AND (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES)) set(CMAKE_JOB_POOL_LINK link_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_LINK ${CMAKE_JOB_POOL_LINK}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_LINK}=${PARALLEL_LINK_JOBS}) From 96c399ff7b623cbebcf7a8e1ae036df5b0f57767 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 11:58:18 +0000 Subject: [PATCH 17/22] Simplify a bit further --- cmake/limit_jobs.cmake | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index b145c069e37..b7cc5cf0967 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -17,7 +17,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) set (PARALLEL_COMPILE_JOBS 1) endif () if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) - set (PARALLEL_COMPILE_JOBS_UNDERUTILIZING TRUE) + message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") endif() endif () @@ -28,7 +28,7 @@ if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) set (PARALLEL_LINK_JOBS 1) endif () if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) - set (PARALLEL_LINK_JOBS_UNDERUTILIZING TRUE) + message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") endif() endif () @@ -45,15 +45,6 @@ endif() message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_COMPILE_JOBS} linker jobs.") -if (PARALLEL_COMPILE_JOBS OR PARALLEL_LINK_JOBS) - if (PARALLEL_COMPILE_JOBS_UNDERUTILIZING) - message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") - endif() - if (PARALLEL_LINK_JOBS_UNDERUTILIZING) - message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") - endif() -endif () - if (PARALLEL_COMPILE_JOBS AND (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) From a2e63404b3ffc9a1a9a5e6380918ab4c033e55a2 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 12:01:17 +0000 Subject: [PATCH 18/22] Further simplify logic --- cmake/limit_jobs.cmake | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index b7cc5cf0967..4d81619aa13 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -45,15 +45,14 @@ endif() message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_COMPILE_JOBS} linker jobs.") -if (PARALLEL_COMPILE_JOBS AND (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES)) +if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_COMPILE ${CMAKE_JOB_POOL_COMPILE}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_COMPILE}=${PARALLEL_COMPILE_JOBS}) endif () -if (PARALLEL_LINK_JOBS AND (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES)) +if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) set(CMAKE_JOB_POOL_LINK link_job_pool${CMAKE_CURRENT_SOURCE_DIR}) string (REGEX REPLACE "[^a-zA-Z0-9]+" "_" CMAKE_JOB_POOL_LINK ${CMAKE_JOB_POOL_LINK}) set_property(GLOBAL APPEND PROPERTY JOB_POOLS ${CMAKE_JOB_POOL_LINK}=${PARALLEL_LINK_JOBS}) endif () - From 8f8cb5eb8289af1f720b976158da364227619250 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 15:48:12 +0200 Subject: [PATCH 19/22] Update cmake/limit_jobs.cmake MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- cmake/limit_jobs.cmake | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 4d81619aa13..39a6f688488 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -43,7 +43,7 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO AND PARALLE set (PARALLEL_LINK_JOBS 2) endif() -message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_COMPILE_JOBS} linker jobs.") +message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs.") if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) From 1dbc88630d56f2b815c2c5746f6706d2ff76a6bc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 17 Jul 2023 17:09:13 +0000 Subject: [PATCH 20/22] Cosmetics --- cmake/limit_jobs.cmake | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/cmake/limit_jobs.cmake b/cmake/limit_jobs.cmake index 39a6f688488..acc38b6fa2a 100644 --- a/cmake/limit_jobs.cmake +++ b/cmake/limit_jobs.cmake @@ -1,7 +1,11 @@ +# Limit compiler/linker job concurrency to avoid OOMs on subtrees where compilation/linking is memory-intensive. +# # Usage from CMake: -# set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") # megabyte -# set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") # megabyte -# include (cmake/limit_jobs.cmake) +# set (MAX_COMPILER_MEMORY 2000 CACHE INTERNAL "") # megabyte +# set (MAX_LINKER_MEMORY 3500 CACHE INTERNAL "") # megabyte +# include (cmake/limit_jobs.cmake) +# +# (bigger values mean fewer jobs) cmake_host_system_information(RESULT TOTAL_PHYSICAL_MEMORY QUERY TOTAL_PHYSICAL_MEMORY) cmake_host_system_information(RESULT NUMBER_OF_LOGICAL_CORES QUERY NUMBER_OF_LOGICAL_CORES) @@ -17,7 +21,7 @@ if (NOT PARALLEL_COMPILE_JOBS AND MAX_COMPILER_MEMORY) set (PARALLEL_COMPILE_JOBS 1) endif () if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) - message(WARNING "The autocalculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") + message(WARNING "The auto-calculated compile jobs limit (${PARALLEL_COMPILE_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_COMPILE_JOBS to override.") endif() endif () @@ -28,7 +32,7 @@ if (NOT PARALLEL_LINK_JOBS AND MAX_LINKER_MEMORY) set (PARALLEL_LINK_JOBS 1) endif () if (PARALLEL_LINK_JOBS LESS NUMBER_OF_LOGICAL_CORES) - message(WARNING "The autocalculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") + message(WARNING "The auto-calculated link jobs limit (${PARALLEL_LINK_JOBS}) underutilizes CPU cores (${NUMBER_OF_LOGICAL_CORES}). Set PARALLEL_LINK_JOBS to override.") endif() endif () @@ -43,7 +47,7 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" AND ENABLE_THINLTO AND PARALLE set (PARALLEL_LINK_JOBS 2) endif() -message(STATUS "System has ${NUMBER_OF_LOGICAL_CORES} logical cores and ${TOTAL_PHYSICAL_MEMORY} megabytes of memory. Building with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs.") +message(STATUS "Building sub-tree with ${PARALLEL_COMPILE_JOBS} compile jobs and ${PARALLEL_LINK_JOBS} linker jobs (system: ${NUMBER_OF_LOGICAL_CORES} cores, ${TOTAL_PHYSICAL_MEMORY} MB DRAM, 'OFF' means the native core count).") if (PARALLEL_COMPILE_JOBS LESS NUMBER_OF_LOGICAL_CORES) set(CMAKE_JOB_POOL_COMPILE compile_job_pool${CMAKE_CURRENT_SOURCE_DIR}) From a955d5621d611dd1f1159a008141a8538575b6db Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 17 Jul 2023 18:53:12 +0000 Subject: [PATCH 21/22] Rename constants in KeyCondition analysis. --- src/Storages/MergeTree/KeyCondition.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index f909d854cf6..3f02a6b197e 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -564,7 +564,17 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( } case (ActionsDAG::ActionType::COLUMN): { - res = &inverted_dag.addColumn({node.column, node.result_type, node.result_name}); + String name; + if (const auto * column_const = typeid_cast(node.column.get())) + /// Re-generate column name for constant. + /// DAG form query (with enabled analyzer) uses suffixes for constants, like 1_UInt8. + /// DAG from PK does not use it. This is breakig match by column name sometimes. + /// Ideally, we should not compare manes, but DAG subtrees instead. + name = ASTLiteral(column_const->getDataColumn()[0]).getColumnName(); + else + name = node.result_name; + + res = &inverted_dag.addColumn({node.column, node.result_type, name}); break; } case (ActionsDAG::ActionType::ALIAS): From dc55c8ed8eeb4891f8ee7161eb962ee991bacd3d Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 18 Jul 2023 09:54:33 +0000 Subject: [PATCH 22/22] Enable no-upgrade-check for 02273_full_sort_join --- tests/queries/0_stateless/02273_full_sort_join.sql.j2 | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 index 8b739330364..43f7354017c 100644 --- a/tests/queries/0_stateless/02273_full_sort_join.sql.j2 +++ b/tests/queries/0_stateless/02273_full_sort_join.sql.j2 @@ -1,4 +1,6 @@ --- Tags: long +-- Tags: long, no-upgrade-check + +-- TODO(@vdimir): remove no-upgrade-check tag after https://github.com/ClickHouse/ClickHouse/pull/51737 is released DROP TABLE IF EXISTS t1; DROP TABLE IF EXISTS t2;