From ea2fd57cca95d45e4d86260f273abaca67012896 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 00:42:59 +0100 Subject: [PATCH 01/46] Set total_rows_approx for trivial queries with LIMIT from system.zeros and generateRandom --- src/Interpreters/InterpreterSelectQuery.cpp | 3 ++ .../QueryPlan/ReadFromSystemNumbersStep.cpp | 6 +-- .../QueryPlan/ReadFromSystemNumbersStep.h | 2 + src/Storages/StorageGenerateRandom.cpp | 15 ++++-- src/Storages/System/StorageSystemNumbers.cpp | 5 +- src/Storages/System/StorageSystemZeros.cpp | 13 +++-- ...rate_random_with_limit_progress_bar.expect | 49 +++++++++++++++++++ ...e_random_with_limit_progress_bar.reference | 0 ...system_zeros_and_generate_random.reference | 0 ...t_for_system_zeros_and_generate_random.sql | 9 ++++ 10 files changed, 86 insertions(+), 16 deletions(-) create mode 100755 tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect create mode 100644 tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference create mode 100644 tests/queries/0_stateless/03024_total_rows_approx_is_set_for_system_zeros_and_generate_random.reference create mode 100644 tests/queries/0_stateless/03024_total_rows_approx_is_set_for_system_zeros_and_generate_random.sql diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 7c87dadfce6a..fa46b1159791 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2501,10 +2501,13 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc max_block_size = std::max(1, max_block_limited); max_threads_execute_query = max_streams = 1; } + if (local_limits.local_limits.size_limits.max_rows != 0) { if (max_block_limited < local_limits.local_limits.size_limits.max_rows) query_info.limit = max_block_limited; + else /// Ask to read just enough rows to make the max_rows limit effective (so it has a chance to be triggered). + query_info.limit = 1 + local_limits.local_limits.size_limits.max_rows; } else { diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp index 7a61d09bdd2d..11371578c79c 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.cpp @@ -12,8 +12,8 @@ #include #include #include -#include "Core/Types.h" -#include "base/types.h" +#include + namespace DB { @@ -443,7 +443,6 @@ Pipe ReadFromSystemNumbersStep::makePipe() /// Build rpn of query filters KeyCondition condition(filter_actions_dag, context, column_names, key_expression); - if (condition.extractPlainRanges(ranges)) { /// Intersect ranges with table range @@ -505,7 +504,6 @@ Pipe ReadFromSystemNumbersStep::makePipe() } } - /// ranges is blank, return a source who has no data if (intersected_ranges.empty()) { diff --git a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.h b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.h index cab0686474ba..bc84e31be626 100644 --- a/src/Processors/QueryPlan/ReadFromSystemNumbersStep.h +++ b/src/Processors/QueryPlan/ReadFromSystemNumbersStep.h @@ -8,6 +8,7 @@ #include #include + namespace DB { @@ -43,4 +44,5 @@ class ReadFromSystemNumbersStep final : public SourceStepWithFilter UInt64 limit; std::shared_ptr storage_limits; }; + } diff --git a/src/Storages/StorageGenerateRandom.cpp b/src/Storages/StorageGenerateRandom.cpp index fbce6c2bb7d1..cdbade51695b 100644 --- a/src/Storages/StorageGenerateRandom.cpp +++ b/src/Storages/StorageGenerateRandom.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -30,12 +31,9 @@ #include #include #include -#include #include -#include - namespace DB { @@ -639,7 +637,7 @@ void registerStorageGenerateRandom(StorageFactory & factory) Pipe StorageGenerateRandom::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & /*query_info*/, + SelectQueryInfo & query_info, ContextPtr context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, @@ -682,7 +680,14 @@ Pipe StorageGenerateRandom::read( pcg64 generate(random_seed); for (UInt64 i = 0; i < num_streams; ++i) - pipes.emplace_back(std::make_shared(max_block_size, max_array_length, max_string_length, generate(), block_header, context)); + { + auto source = std::make_shared(max_block_size, max_array_length, max_string_length, generate(), block_header, context); + + if (i == 0 && query_info.limit) + source->addTotalRowsApprox(query_info.limit); + + pipes.emplace_back(std::move(source)); + } return Pipe::unitePipes(std::move(pipes)); } diff --git a/src/Storages/System/StorageSystemNumbers.cpp b/src/Storages/System/StorageSystemNumbers.cpp index 629b11ee7f1f..da700a7a4e9d 100644 --- a/src/Storages/System/StorageSystemNumbers.cpp +++ b/src/Storages/System/StorageSystemNumbers.cpp @@ -1,17 +1,14 @@ #include #include -#include #include #include -#include #include #include #include -#include -#include #include + namespace DB { diff --git a/src/Storages/System/StorageSystemZeros.cpp b/src/Storages/System/StorageSystemZeros.cpp index ce2e153ea666..a48b109fbbe5 100644 --- a/src/Storages/System/StorageSystemZeros.cpp +++ b/src/Storages/System/StorageSystemZeros.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -6,6 +7,7 @@ #include #include + namespace DB { @@ -93,7 +95,7 @@ StorageSystemZeros::StorageSystemZeros(const StorageID & table_id_, bool multith Pipe StorageSystemZeros::read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo &, + SelectQueryInfo & query_info, ContextPtr /*context*/, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, @@ -123,8 +125,13 @@ Pipe StorageSystemZeros::read( { auto source = std::make_shared(max_block_size, limit ? *limit : 0, state); - if (limit && i == 0) - source->addTotalRowsApprox(*limit); + if (i == 0) + { + if (limit) + source->addTotalRowsApprox(*limit); + else if (query_info.limit) + source->addTotalRowsApprox(query_info.limit); + } res.addSource(std::move(source)); } diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect new file mode 100755 index 000000000000..272dc0fdfef7 --- /dev/null +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect @@ -0,0 +1,49 @@ +#!/usr/bin/expect -f + +set basedir [file dirname $argv0] +set basename [file tail $argv0] +if {[info exists env(CLICKHOUSE_TMP)]} { + set CLICKHOUSE_TMP $env(CLICKHOUSE_TMP) +} else { + set CLICKHOUSE_TMP "." +} +exp_internal -f $CLICKHOUSE_TMP/$basename.debuglog 0 + +log_user 0 +set timeout 60 +match_max 100000 +set stty_init "rows 25 cols 120" + +expect_after { + -i $any_spawn_id eof { exp_continue } + -i $any_spawn_id timeout { exit 1 } +} + +spawn clickhouse-local +expect ":) " + +# Trivial SELECT with LIMIT from system.zeros shows progress bar. +send "SELECT * FROM system.zeros LIMIT 1e11 FORMAT Null\r" +expect "Progress: " +expect "█" +send "\3" +expect "Query was cancelled." +expect ":) " + +send "SELECT * FROM system.zeros_mt LIMIT 1e11 FORMAT Null\r" +expect "Progress: " +expect "█" +send "\3" +expect "Query was cancelled." +expect ":) " + +# As well as from generateRandom +send "SELECT * FROM generateRandom() LIMIT 1e9 FORMAT Null\r" +expect "Progress: " +expect "█" +send "\3" +expect "Query was cancelled." +expect ":) " + +send "exit\r" +expect eof diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03024_total_rows_approx_is_set_for_system_zeros_and_generate_random.reference b/tests/queries/0_stateless/03024_total_rows_approx_is_set_for_system_zeros_and_generate_random.reference new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/tests/queries/0_stateless/03024_total_rows_approx_is_set_for_system_zeros_and_generate_random.sql b/tests/queries/0_stateless/03024_total_rows_approx_is_set_for_system_zeros_and_generate_random.sql new file mode 100644 index 000000000000..0db09ead2cda --- /dev/null +++ b/tests/queries/0_stateless/03024_total_rows_approx_is_set_for_system_zeros_and_generate_random.sql @@ -0,0 +1,9 @@ +SET max_rows_to_read = 1e11; + +SELECT * FROM system.numbers LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS } +SELECT * FROM system.numbers_mt LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS } + +SELECT * FROM system.zeros LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS } +SELECT * FROM system.zeros_mt LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS } + +SELECT * FROM generateRandom() LIMIT 1e12 FORMAT Null; -- { serverError TOO_MANY_ROWS } From 86039802e3327c0a527135233f3589d6bcea2348 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 00:46:48 +0100 Subject: [PATCH 02/46] Set total_rows_approx for trivial queries with LIMIT from system.zeros and generateRandom --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index fa46b1159791..65beef27d161 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2506,7 +2506,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc { if (max_block_limited < local_limits.local_limits.size_limits.max_rows) query_info.limit = max_block_limited; - else /// Ask to read just enough rows to make the max_rows limit effective (so it has a chance to be triggered). + else if (local_limits.local_limits.size_limits.max_rows < std::numeric_limits::max()) /// Ask to read just enough rows to make the max_rows limit effective (so it has a chance to be triggered). query_info.limit = 1 + local_limits.local_limits.size_limits.max_rows; } else From c55e45bff6212e8e4828198fd13a56bc174d2062 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 24 Mar 2024 18:58:57 +0100 Subject: [PATCH 03/46] Test robustness --- ...023_zeros_generate_random_with_limit_progress_bar.expect | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect index 272dc0fdfef7..de15a1991328 100755 --- a/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect +++ b/tests/queries/0_stateless/03023_zeros_generate_random_with_limit_progress_bar.expect @@ -23,14 +23,14 @@ spawn clickhouse-local expect ":) " # Trivial SELECT with LIMIT from system.zeros shows progress bar. -send "SELECT * FROM system.zeros LIMIT 1e11 FORMAT Null\r" +send "SELECT * FROM system.zeros LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" expect "Progress: " expect "█" send "\3" expect "Query was cancelled." expect ":) " -send "SELECT * FROM system.zeros_mt LIMIT 1e11 FORMAT Null\r" +send "SELECT * FROM system.zeros_mt LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" expect "Progress: " expect "█" send "\3" @@ -38,7 +38,7 @@ expect "Query was cancelled." expect ":) " # As well as from generateRandom -send "SELECT * FROM generateRandom() LIMIT 1e9 FORMAT Null\r" +send "SELECT * FROM generateRandom() LIMIT 10000000 FORMAT Null SETTINGS max_execution_speed = 1000000, timeout_before_checking_execution_speed = 0, max_block_size = 128\r" expect "Progress: " expect "█" send "\3" From 382e4fba5c98f3d647356791d2e4e4c8da57f802 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Gr=C3=A9goire=20Pineau?= Date: Wed, 24 Apr 2024 11:04:56 +0200 Subject: [PATCH 04/46] dx: Enhance error message when non-deterministic function is used with Replicated source --- src/Interpreters/MutationsInterpreter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 35fd549559b3..667a3e2e7a6a 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1313,7 +1313,7 @@ void MutationsInterpreter::validate() if (nondeterministic_func_data.nondeterministic_function_name) throw Exception(ErrorCodes::BAD_ARGUMENTS, - "ALTER UPDATE/ALTER DELETE statements must use only deterministic functions. " + "The source storage is replicated so ALTER UPDATE/ALTER DELETE statements must use only deterministic functions. " "Function '{}' is non-deterministic", *nondeterministic_func_data.nondeterministic_function_name); } } From 4e6e234d24eb46d62c7e7f0e99a1ecccda5cc0f4 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 3 May 2024 12:19:57 +0200 Subject: [PATCH 05/46] Safer parent part access --- src/Storages/MergeTree/MergeTreeReadPoolBase.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 0cbb0a86b2f4..3e10285d6b0b 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -113,9 +113,9 @@ MergeTreeReadTaskPtr MergeTreeReadPoolBase::createTask( ? std::make_unique(*read_info->shared_size_predictor) : nullptr; /// make a copy - auto get_part_name = [](const auto & task_info) -> const String & + auto get_part_name = [](const auto & task_info) -> String { - return task_info.data_part->isProjectionPart() ? task_info.data_part->getParentPart()->name : task_info.data_part->name; + return task_info.data_part->isProjectionPart() ? task_info.data_part->getParentPartName() : task_info.data_part->name; }; auto extras = getExtras(); From 314573b7a66a2379aecb0b8e680d24561f00e03e Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 3 May 2024 12:40:05 +0200 Subject: [PATCH 06/46] Add debug check --- .../MergeTree/MergeTreeReadPoolBase.cpp | 23 ++++++++++++++++++- 1 file changed, 22 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 3e10285d6b0b..c759a12e151f 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -6,6 +6,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + MergeTreeReadPoolBase::MergeTreeReadPoolBase( RangesInDataParts && parts_, VirtualFields shared_virtual_fields_, @@ -115,7 +120,23 @@ MergeTreeReadTaskPtr MergeTreeReadPoolBase::createTask( auto get_part_name = [](const auto & task_info) -> String { - return task_info.data_part->isProjectionPart() ? task_info.data_part->getParentPartName() : task_info.data_part->name; + const auto & data_part = task_info.data_part; + + if (data_part->isProjectionPart()) + { + auto parent_part_name = data_part->getParentPartName(); + + auto parent_part = data_part->storage.getPartIfExists( + parent_part_name, {MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); + + if (!parent_part) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Did not find parent part {} for potentially broken projection part {}", + parent_part_name, data_part->getDataPartStorage().getFullPath()); + + return parent_part_name; + } + + return data_part->name; }; auto extras = getExtras(); From ab7d457c51d92123d1845486a2f3243324f06503 Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Mon, 6 May 2024 11:35:48 +0200 Subject: [PATCH 07/46] Update ErrorCodes for functions using NUMBER_OF_ARGUMENTS_DOESNT_MATCH --- src/AggregateFunctions/AggregateFunctionAggThrow.cpp | 4 ++-- .../AggregateFunctionGroupArrayInsertAt.cpp | 3 ++- src/AggregateFunctions/AggregateFunctionMLMethod.cpp | 7 ++++--- src/AggregateFunctions/AggregateFunctionMannWhitney.cpp | 4 ++-- src/AggregateFunctions/AggregateFunctionQuantile.cpp | 4 ++-- .../AggregateFunctionQuantileBFloat16Weighted.cpp | 4 ++-- src/AggregateFunctions/AggregateFunctionQuantileDD.cpp | 4 ++-- .../AggregateFunctionQuantileExactHigh.cpp | 4 ++-- .../AggregateFunctionQuantileExactInclusive.cpp | 4 ++-- .../AggregateFunctionQuantileTiming.cpp | 4 ++-- src/AggregateFunctions/AggregateFunctionTopK.cpp | 4 ++-- src/AggregateFunctions/AggregateFunctionWelchTTest.cpp | 4 ++-- .../Combinators/AggregateFunctionArray.cpp | 4 ++-- src/AggregateFunctions/Combinators/AggregateFunctionIf.h | 4 ++-- src/Functions/Kusto/KqlArraySort.cpp | 4 ++-- src/Functions/MultiSearchFirstPositionImpl.h | 4 ++-- src/Functions/MultiSearchImpl.h | 4 ++-- src/Functions/array/arrayEnumerateExtended.h | 4 ++-- src/Functions/array/arrayEnumerateRanked.h | 7 ++++--- src/Functions/array/arrayFold.cpp | 6 +++--- src/Functions/array/arrayReduce.cpp | 4 ++-- src/Functions/array/arrayZip.cpp | 4 ++-- src/Functions/concat.cpp | 6 +++--- src/Functions/generateULID.cpp | 4 ++-- src/Functions/jsonMergePatch.cpp | 4 ++-- src/Functions/nested.cpp | 9 +++++---- src/Functions/now64.cpp | 4 ++-- src/Functions/nowInBlock.cpp | 4 ++-- src/Functions/parseTimeDelta.cpp | 7 ++++--- src/Functions/pointInPolygon.cpp | 4 ++-- src/TableFunctions/TableFunctionValues.cpp | 4 ++-- 31 files changed, 73 insertions(+), 68 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAggThrow.cpp b/src/AggregateFunctions/AggregateFunctionAggThrow.cpp index 27432bc94ba0..ebf6cabbcfd5 100644 --- a/src/AggregateFunctions/AggregateFunctionAggThrow.cpp +++ b/src/AggregateFunctions/AggregateFunctionAggThrow.cpp @@ -16,7 +16,7 @@ struct Settings; namespace ErrorCodes { extern const int AGGREGATE_FUNCTION_THROW; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } namespace @@ -116,7 +116,7 @@ void registerAggregateFunctionAggThrow(AggregateFunctionFactory & factory) if (parameters.size() == 1) throw_probability = parameters[0].safeGet(); else if (parameters.size() > 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} cannot have more than one parameter", name); + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} cannot have more than one parameter", name); return std::make_shared(argument_types, parameters, throw_probability); }); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.cpp b/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.cpp index 60e8df642839..8674aac2e907 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.cpp @@ -27,6 +27,7 @@ struct Settings; namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; extern const int TOO_LARGE_ARRAY_SIZE; extern const int CANNOT_CONVERT_TYPE; extern const int ILLEGAL_TYPE_OF_ARGUMENT; @@ -74,7 +75,7 @@ class AggregateFunctionGroupArrayInsertAtGeneric final if (!params.empty()) { if (params.size() > 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at most two parameters.", getName()); + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at most two parameters.", getName()); default_value = params[0]; diff --git a/src/AggregateFunctions/AggregateFunctionMLMethod.cpp b/src/AggregateFunctions/AggregateFunctionMLMethod.cpp index d2178437770b..730b5cfa5ccc 100644 --- a/src/AggregateFunctions/AggregateFunctionMLMethod.cpp +++ b/src/AggregateFunctions/AggregateFunctionMLMethod.cpp @@ -22,7 +22,8 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } namespace @@ -34,12 +35,12 @@ namespace const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { if (parameters.size() > 4) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at most four parameters: " "learning_rate, l2_regularization_coef, mini-batch size and weights_updater method", name); if (argument_types.size() < 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least two arguments: target and model's parameters", name); for (size_t i = 0; i < argument_types.size(); ++i) diff --git a/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp b/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp index e7bc5df335f7..d185058dbd06 100644 --- a/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp +++ b/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp @@ -21,7 +21,7 @@ namespace ErrorCodes { extern const int NOT_IMPLEMENTED; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; extern const int BAD_ARGUMENTS; } @@ -141,7 +141,7 @@ class AggregateFunctionMannWhitney final: : IAggregateFunctionDataHelper ({arguments}, {}, createResultType()) { if (params.size() > 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} require two parameter or less", getName()); + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} require two parameter or less", getName()); if (params.empty()) { diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.cpp b/src/AggregateFunctions/AggregateFunctionQuantile.cpp index 4d37ec69d26a..f72b28030c35 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantile.cpp @@ -14,7 +14,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NOT_IMPLEMENTED; } @@ -118,7 +118,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile( const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) { if (argument_types.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name); const DataTypePtr & argument_type = argument_types[0]; WhichDataType which(argument_type); diff --git a/src/AggregateFunctions/AggregateFunctionQuantileBFloat16Weighted.cpp b/src/AggregateFunctions/AggregateFunctionQuantileBFloat16Weighted.cpp index 6d881b77c160..b0a39b1fdabe 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantileBFloat16Weighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantileBFloat16Weighted.cpp @@ -12,7 +12,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -27,7 +27,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile( const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) { if (argument_types.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name); const DataTypePtr & argument_type = argument_types[0]; WhichDataType which(argument_type); diff --git a/src/AggregateFunctions/AggregateFunctionQuantileDD.cpp b/src/AggregateFunctions/AggregateFunctionQuantileDD.cpp index f3d6b26ee759..7b1ae43038ae 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantileDD.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantileDD.cpp @@ -13,7 +13,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -29,7 +29,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile( const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) { if (argument_types.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name); const DataTypePtr & argument_type = argument_types[0]; WhichDataType which(argument_type); diff --git a/src/AggregateFunctions/AggregateFunctionQuantileExactHigh.cpp b/src/AggregateFunctions/AggregateFunctionQuantileExactHigh.cpp index b44df755158c..6fa4158076c6 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantileExactHigh.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantileExactHigh.cpp @@ -13,7 +13,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -29,7 +29,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile( const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) { if (argument_types.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name); const DataTypePtr & argument_type = argument_types[0]; WhichDataType which(argument_type); diff --git a/src/AggregateFunctions/AggregateFunctionQuantileExactInclusive.cpp b/src/AggregateFunctions/AggregateFunctionQuantileExactInclusive.cpp index d8287be86cad..bed5cf95bd8e 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantileExactInclusive.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantileExactInclusive.cpp @@ -13,7 +13,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -28,7 +28,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile( const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) { if (argument_types.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name); const DataTypePtr & argument_type = argument_types[0]; WhichDataType which(argument_type); diff --git a/src/AggregateFunctions/AggregateFunctionQuantileTiming.cpp b/src/AggregateFunctions/AggregateFunctionQuantileTiming.cpp index 1bb77892c0d3..e293fc6e59d6 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantileTiming.cpp +++ b/src/AggregateFunctions/AggregateFunctionQuantileTiming.cpp @@ -13,7 +13,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -28,7 +28,7 @@ AggregateFunctionPtr createAggregateFunctionQuantile( const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) { if (argument_types.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires at least one argument", name); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires at least one argument", name); const DataTypePtr & argument_type = argument_types[0]; WhichDataType which(argument_type); diff --git a/src/AggregateFunctions/AggregateFunctionTopK.cpp b/src/AggregateFunctions/AggregateFunctionTopK.cpp index d0e4f507d460..26f756abe180 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.cpp +++ b/src/AggregateFunctions/AggregateFunctionTopK.cpp @@ -35,7 +35,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } @@ -467,7 +467,7 @@ AggregateFunctionPtr createAggregateFunctionTopK(const std::string & name, const if (!params.empty()) { if (params.size() > 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function '{}' requires three parameters or less", name); threshold = applyVisitor(FieldVisitorConvertToNumber(), params[0]); diff --git a/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp b/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp index 849f81279e72..7f1c4861fbc4 100644 --- a/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp +++ b/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp @@ -7,7 +7,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } namespace DB @@ -80,7 +80,7 @@ AggregateFunctionPtr createAggregateFunctionWelchTTest( assertBinary(name, argument_types); if (parameters.size() > 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Aggregate function {} requires zero or one parameter.", name); + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Aggregate function {} requires zero or one parameter.", name); if (!isNumber(argument_types[0]) || !isNumber(argument_types[1])) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} only supports numerical types", name); diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp b/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp index a54092f0bcb7..6b60faadf073 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp +++ b/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp @@ -9,7 +9,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -26,7 +26,7 @@ class AggregateFunctionCombinatorArray final : public IAggregateFunctionCombinat DataTypes transformArguments(const DataTypes & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "-Array aggregate functions require at least one argument"); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "-Array aggregate functions require at least one argument"); DataTypes nested_arguments; for (const auto & type : arguments) diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionIf.h b/src/AggregateFunctions/Combinators/AggregateFunctionIf.h index a893fc91780a..f57cfa41752b 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionIf.h +++ b/src/AggregateFunctions/Combinators/AggregateFunctionIf.h @@ -18,7 +18,7 @@ struct Settings; namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -42,7 +42,7 @@ class AggregateFunctionIf final : public IAggregateFunctionHelperonlyNull(); diff --git a/src/Functions/Kusto/KqlArraySort.cpp b/src/Functions/Kusto/KqlArraySort.cpp index 5be36328cc37..ac301c474e89 100644 --- a/src/Functions/Kusto/KqlArraySort.cpp +++ b/src/Functions/Kusto/KqlArraySort.cpp @@ -11,7 +11,7 @@ namespace DB { namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; } @@ -35,7 +35,7 @@ class FunctionKqlArraySort : public KqlFunctionBase { if (arguments.empty()) throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} needs at least one argument; passed {}.", getName(), arguments.size()); diff --git a/src/Functions/MultiSearchFirstPositionImpl.h b/src/Functions/MultiSearchFirstPositionImpl.h index 99dd3f9d3940..aca96cabf045 100644 --- a/src/Functions/MultiSearchFirstPositionImpl.h +++ b/src/Functions/MultiSearchFirstPositionImpl.h @@ -10,7 +10,7 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } template @@ -37,7 +37,7 @@ struct MultiSearchFirstPositionImpl { // For performance of Volnitsky search, it is crucial to save only one byte for pattern number. if (needles_arr.size() > std::numeric_limits::max()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at most {}", name, std::to_string(needles_arr.size()), std::to_string(std::numeric_limits::max())); diff --git a/src/Functions/MultiSearchImpl.h b/src/Functions/MultiSearchImpl.h index fb7d56f302ab..d3d1680481f1 100644 --- a/src/Functions/MultiSearchImpl.h +++ b/src/Functions/MultiSearchImpl.h @@ -10,7 +10,7 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } template @@ -37,7 +37,7 @@ struct MultiSearchImpl { // For performance of Volnitsky search, it is crucial to save only one byte for pattern number. if (needles_arr.size() > std::numeric_limits::max()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at most {}", name, needles_arr.size(), std::to_string(std::numeric_limits::max())); diff --git a/src/Functions/array/arrayEnumerateExtended.h b/src/Functions/array/arrayEnumerateExtended.h index cf38afcfa5af..67c0981108fd 100644 --- a/src/Functions/array/arrayEnumerateExtended.h +++ b/src/Functions/array/arrayEnumerateExtended.h @@ -18,7 +18,7 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int SIZES_OF_ARRAYS_DONT_MATCH; @@ -43,7 +43,7 @@ class FunctionArrayEnumerateExtended : public IFunction DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at least 1.", getName(), arguments.size()); diff --git a/src/Functions/array/arrayEnumerateRanked.h b/src/Functions/array/arrayEnumerateRanked.h index 04fa305368de..ad325fe542ab 100644 --- a/src/Functions/array/arrayEnumerateRanked.h +++ b/src/Functions/array/arrayEnumerateRanked.h @@ -59,7 +59,8 @@ namespace DB { namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int SIZES_OF_ARRAYS_DONT_MATCH; } @@ -101,7 +102,7 @@ class FunctionArrayEnumerateRankedExtended : public IFunction DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at least 1.", getName(), arguments.size()); @@ -238,7 +239,7 @@ ColumnPtr FunctionArrayEnumerateRankedExtended::executeImpl( } if (offsets_by_depth.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "No arrays passed to function {}", getName()); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No arrays passed to function {}", getName()); auto res_nested = ColumnUInt32::create(); diff --git a/src/Functions/array/arrayFold.cpp b/src/Functions/array/arrayFold.cpp index 63c14f475fc3..5170c9a5b5fc 100644 --- a/src/Functions/array/arrayFold.cpp +++ b/src/Functions/array/arrayFold.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int SIZES_OF_ARRAYS_DONT_MATCH; extern const int TYPE_MISMATCH; } @@ -41,7 +41,7 @@ class FunctionArrayFold : public IFunction void getLambdaArgumentTypes(DataTypes & arguments) const override { if (arguments.size() < 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires as arguments a lambda function, at least one array and an accumulator", getName()); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} requires as arguments a lambda function, at least one array and an accumulator", getName()); DataTypes accumulator_and_array_types(arguments.size() - 1); accumulator_and_array_types[0] = arguments.back(); @@ -64,7 +64,7 @@ class FunctionArrayFold : public IFunction DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.size() < 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires as arguments a lambda function, at least one array and an accumulator", getName()); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} requires as arguments a lambda function, at least one array and an accumulator", getName()); const auto * lambda_function_type = checkAndGetDataType(arguments[0].type.get()); if (!lambda_function_type) diff --git a/src/Functions/array/arrayReduce.cpp b/src/Functions/array/arrayReduce.cpp index d47d1ae98ccd..d70b8b402338 100644 --- a/src/Functions/array/arrayReduce.cpp +++ b/src/Functions/array/arrayReduce.cpp @@ -21,7 +21,7 @@ namespace DB namespace ErrorCodes { extern const int SIZES_OF_ARRAYS_DONT_MATCH; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int BAD_ARGUMENTS; @@ -73,7 +73,7 @@ DataTypePtr FunctionArrayReduce::getReturnTypeImpl(const ColumnsWithTypeAndName /// (possibly with parameters in parentheses, for example: "quantile(0.99)"). if (arguments.size() < 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at least 2.", getName(), arguments.size()); diff --git a/src/Functions/array/arrayZip.cpp b/src/Functions/array/arrayZip.cpp index 44c323e3fe31..6c6fff5926bb 100644 --- a/src/Functions/array/arrayZip.cpp +++ b/src/Functions/array/arrayZip.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int SIZES_OF_ARRAYS_DONT_MATCH; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_COLUMN; } @@ -39,7 +39,7 @@ class FunctionArrayZip : public IFunction DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} needs at least one argument; passed {}." , getName(), arguments.size()); DataTypes arguments_types; diff --git a/src/Functions/concat.cpp b/src/Functions/concat.cpp index c75a806559c0..68cfcdb8d90b 100644 --- a/src/Functions/concat.cpp +++ b/src/Functions/concat.cpp @@ -16,7 +16,7 @@ namespace DB { namespace ErrorCodes { -extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; } using namespace GatherUtils; @@ -48,7 +48,7 @@ class ConcatImpl : public IFunction { if (arguments.size() < 2) throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at least 2", getName(), arguments.size()); @@ -225,7 +225,7 @@ class ConcatOverloadResolver : public IFunctionOverloadResolver { if (arguments.empty()) throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at least 1.", getName(), arguments.size()); diff --git a/src/Functions/generateULID.cpp b/src/Functions/generateULID.cpp index 9c5c9403185a..f2f2d8ae3b9f 100644 --- a/src/Functions/generateULID.cpp +++ b/src/Functions/generateULID.cpp @@ -17,7 +17,7 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } class FunctionGenerateULID : public IFunction @@ -45,7 +45,7 @@ class FunctionGenerateULID : public IFunction { if (arguments.size() > 1) throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be 0 or 1.", getName(), arguments.size()); diff --git a/src/Functions/jsonMergePatch.cpp b/src/Functions/jsonMergePatch.cpp index 65946721432d..a83daacdbf64 100644 --- a/src/Functions/jsonMergePatch.cpp +++ b/src/Functions/jsonMergePatch.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int ILLEGAL_COLUMN; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -53,7 +53,7 @@ namespace DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least one argument.", getName()); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} requires at least one argument.", getName()); for (const auto & arg : arguments) if (!isString(arg.type)) diff --git a/src/Functions/nested.cpp b/src/Functions/nested.cpp index 679bb4f73d8a..bdaf57d65c9e 100644 --- a/src/Functions/nested.cpp +++ b/src/Functions/nested.cpp @@ -18,9 +18,10 @@ namespace DB namespace ErrorCodes { + extern const int BAD_ARGUMENTS; extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int SIZES_OF_ARRAYS_DONT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; } namespace @@ -64,19 +65,19 @@ class FunctionNested : public IFunction { size_t arguments_size = arguments.size(); if (arguments_size < 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be at least 2", getName(), arguments_size); Names nested_names = extractNestedNames(arguments[0].column); if (nested_names.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "First argument for function {} must be constant column with array of strings", getName()); if (nested_names.size() != arguments_size - 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Size of nested names array for function {} does not match arrays arguments size. Actual {}. Expected {}", getName(), nested_names.size(), diff --git a/src/Functions/now64.cpp b/src/Functions/now64.cpp index 0f1e8a042363..d6f8474c9848 100644 --- a/src/Functions/now64.cpp +++ b/src/Functions/now64.cpp @@ -18,7 +18,7 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int CANNOT_CLOCK_GETTIME; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } namespace @@ -128,7 +128,7 @@ class Now64OverloadResolver : public IFunctionOverloadResolver if (arguments.size() > 2) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Arguments size of function {} should be 0, or 1, or 2", getName()); + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Arguments size of function {} should be 0, or 1, or 2", getName()); } if (!arguments.empty()) { diff --git a/src/Functions/nowInBlock.cpp b/src/Functions/nowInBlock.cpp index 0d5f9c457804..74f420986c81 100644 --- a/src/Functions/nowInBlock.cpp +++ b/src/Functions/nowInBlock.cpp @@ -12,7 +12,7 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; } @@ -63,7 +63,7 @@ class FunctionNowInBlock : public IFunction { if (arguments.size() > 1) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Arguments size of function {} should be 0 or 1", getName()); + throw Exception(ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Arguments size of function {} should be 0 or 1", getName()); } if (arguments.size() == 1 && !isStringOrFixedString(arguments[0].type)) { diff --git a/src/Functions/parseTimeDelta.cpp b/src/Functions/parseTimeDelta.cpp index 7743a0cb6640..44eeb1a289f9 100644 --- a/src/Functions/parseTimeDelta.cpp +++ b/src/Functions/parseTimeDelta.cpp @@ -11,7 +11,8 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; + extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int BAD_ARGUMENTS; } @@ -117,14 +118,14 @@ namespace { if (arguments.empty()) throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be 1.", getName(), arguments.size()); if (arguments.size() > 1) throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + ErrorCodes::TOO_MANY_ARGUMENTS_FOR_FUNCTION, "Number of arguments for function {} doesn't match: passed {}, should be 1.", getName(), arguments.size()); diff --git a/src/Functions/pointInPolygon.cpp b/src/Functions/pointInPolygon.cpp index 0e4467a82109..55f89b71d6ff 100644 --- a/src/Functions/pointInPolygon.cpp +++ b/src/Functions/pointInPolygon.cpp @@ -37,7 +37,7 @@ namespace DB { namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int BAD_ARGUMENTS; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int ILLEGAL_COLUMN; @@ -87,7 +87,7 @@ class FunctionPointInPolygon : public IFunction { if (arguments.size() < 2) { - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} requires at least 2 arguments", getName()); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} requires at least 2 arguments", getName()); } /** We allow function invocation in one of the following forms: diff --git a/src/TableFunctions/TableFunctionValues.cpp b/src/TableFunctions/TableFunctionValues.cpp index 7b2a61c25eb3..4b56fa57091d 100644 --- a/src/TableFunctions/TableFunctionValues.cpp +++ b/src/TableFunctions/TableFunctionValues.cpp @@ -25,7 +25,7 @@ namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } @@ -112,7 +112,7 @@ void TableFunctionValues::parseArguments(const ASTPtr & ast_function, ContextPtr ASTs & args = args_func.at(0)->children; if (args.empty()) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires at least 1 argument", getName()); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Table function '{}' requires at least 1 argument", getName()); const auto & literal = args[0]->as(); String value; From 9d55bc82d6609633a135a5044f05aadeaff21755 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 May 2024 16:19:57 +0200 Subject: [PATCH 08/46] Revert "Revert "Do not remove server constants from GROUP BY key for secondary query."" --- src/Planner/PlannerExpressionAnalysis.cpp | 6 ++++-- .../03095_group_by_server_constants_bug.reference | 1 + .../0_stateless/03095_group_by_server_constants_bug.sql | 5 +++++ 3 files changed, 10 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03095_group_by_server_constants_bug.reference create mode 100644 tests/queries/0_stateless/03095_group_by_server_constants_bug.sql diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index ad8db83d66c4..d7fa270a6430 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -85,6 +85,8 @@ std::optional analyzeAggregation(const QueryTreeNodeP bool group_by_use_nulls = planner_context->getQueryContext()->getSettingsRef().group_by_use_nulls && (query_node.isGroupByWithGroupingSets() || query_node.isGroupByWithRollup() || query_node.isGroupByWithCube()); + bool is_secondary_query = planner_context->getQueryContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; + if (query_node.hasGroupBy()) { if (query_node.isGroupByWithGroupingSets()) @@ -100,7 +102,7 @@ std::optional analyzeAggregation(const QueryTreeNodeP auto is_constant_key = grouping_set_key_node->as() != nullptr; group_by_with_constant_keys |= is_constant_key; - if (is_constant_key && !aggregates_descriptions.empty()) + if (!is_secondary_query && is_constant_key && !aggregates_descriptions.empty()) continue; auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, grouping_set_key_node); @@ -152,7 +154,7 @@ std::optional analyzeAggregation(const QueryTreeNodeP auto is_constant_key = group_by_key_node->as() != nullptr; group_by_with_constant_keys |= is_constant_key; - if (is_constant_key && !aggregates_descriptions.empty()) + if (!is_secondary_query && is_constant_key && !aggregates_descriptions.empty()) continue; auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, group_by_key_node); diff --git a/tests/queries/0_stateless/03095_group_by_server_constants_bug.reference b/tests/queries/0_stateless/03095_group_by_server_constants_bug.reference new file mode 100644 index 000000000000..80ab3c879bb0 --- /dev/null +++ b/tests/queries/0_stateless/03095_group_by_server_constants_bug.reference @@ -0,0 +1 @@ +r1 2 diff --git a/tests/queries/0_stateless/03095_group_by_server_constants_bug.sql b/tests/queries/0_stateless/03095_group_by_server_constants_bug.sql new file mode 100644 index 000000000000..9f9fda1ef620 --- /dev/null +++ b/tests/queries/0_stateless/03095_group_by_server_constants_bug.sql @@ -0,0 +1,5 @@ +SELECT serverUUID() AS s, count() FROM remote('127.0.0.{1,2}', system.one) GROUP BY s format Null; + +select getMacro('replica') as s, count() from remote('127.0.0.{1,2}', system.one) group by s; + +select uptime() as s, count() FROM remote('127.0.0.{1,2}', system.one) group by s format Null; From 670c8c11886ee5763639bccb28cf45f74f683c05 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Mon, 6 May 2024 21:42:25 +0200 Subject: [PATCH 09/46] fix exception --- src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp b/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp index 6b60faadf073..16646c702bf9 100644 --- a/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp +++ b/src/AggregateFunctions/Combinators/AggregateFunctionArray.cpp @@ -26,7 +26,7 @@ class AggregateFunctionCombinatorArray final : public IAggregateFunctionCombinat DataTypes transformArguments(const DataTypes & arguments) const override { if (arguments.empty()) - throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "-Array aggregate functions require at least one argument"); + throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Array aggregate functions require at least one argument"); DataTypes nested_arguments; for (const auto & type : arguments) From 9fcb5d2ebfb3e2105b5d6171a1a53ff75c0023f4 Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Mon, 6 May 2024 23:44:27 +0200 Subject: [PATCH 10/46] Move ErrorCodes inside DB namespace --- .../AggregateFunctionMannWhitney.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp b/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp index d185058dbd06..8434f11eae6b 100644 --- a/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp +++ b/src/AggregateFunctions/AggregateFunctionMannWhitney.cpp @@ -17,6 +17,11 @@ #include +namespace DB +{ + +struct Settings; + namespace ErrorCodes { extern const int NOT_IMPLEMENTED; @@ -25,11 +30,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -namespace DB -{ - -struct Settings; - namespace { From 4703eb943b9586be7cab2429a170d9cf9131558f Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Mon, 6 May 2024 23:59:59 +0200 Subject: [PATCH 11/46] Move ErrorCodes inside DB namespace --- src/AggregateFunctions/AggregateFunctionWelchTTest.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp b/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp index 7f1c4861fbc4..7efd61542272 100644 --- a/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp +++ b/src/AggregateFunctions/AggregateFunctionWelchTTest.cpp @@ -4,16 +4,16 @@ #include +namespace DB +{ +struct Settings; + namespace ErrorCodes { extern const int BAD_ARGUMENTS; extern const int TOO_MANY_ARGUMENTS_FOR_FUNCTION; } -namespace DB -{ -struct Settings; - namespace { From 5e5477302a71e8959feb8daab1b9459e16b2e168 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 7 May 2024 09:31:39 +0200 Subject: [PATCH 12/46] Update src/Storages/MergeTree/MergeTreeReadPoolBase.cpp Co-authored-by: Alexander Tokmakov --- src/Storages/MergeTree/MergeTreeReadPoolBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index c759a12e151f..36673238f3ba 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -130,7 +130,7 @@ MergeTreeReadTaskPtr MergeTreeReadPoolBase::createTask( parent_part_name, {MergeTreeDataPartState::PreActive, MergeTreeDataPartState::Active, MergeTreeDataPartState::Outdated}); if (!parent_part) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Did not find parent part {} for potentially broken projection part {}", + throw Exception(ErrorCodes::LOGICAL_ERROR, "Did not find parent part {} for projection part {}", parent_part_name, data_part->getDataPartStorage().getFullPath()); return parent_part_name; From 49284724ef4b34bbc5112bfa63a99625e8d35fe7 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 7 May 2024 16:10:21 +0200 Subject: [PATCH 13/46] fix errorCodes in fast tests --- tests/queries/0_stateless/00909_arrayEnumerateUniq.sql | 2 +- tests/queries/0_stateless/01045_array_zip.sql | 2 +- tests/queries/0_stateless/02354_parse_timedelta.sql | 4 ++-- tests/queries/0_stateless/02372_now_in_block.sql | 2 +- tests/queries/0_stateless/02718_array_fold.sql | 8 ++++---- 5 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/00909_arrayEnumerateUniq.sql b/tests/queries/0_stateless/00909_arrayEnumerateUniq.sql index 0bdb338e9d27..fe01b2185c2a 100644 --- a/tests/queries/0_stateless/00909_arrayEnumerateUniq.sql +++ b/tests/queries/0_stateless/00909_arrayEnumerateUniq.sql @@ -152,7 +152,7 @@ DROP TABLE arrays_test; select '---------BAD'; -SELECT arrayEnumerateUniqRanked(); -- { serverError 42 } +SELECT arrayEnumerateUniqRanked(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } SELECT arrayEnumerateUniqRanked([]); SELECT arrayEnumerateUniqRanked(1); -- { serverError 36 } SELECT arrayEnumerateUniqRanked(2,[]); -- { serverError 36 } diff --git a/tests/queries/0_stateless/01045_array_zip.sql b/tests/queries/0_stateless/01045_array_zip.sql index 1a85e6a0874b..a2d54c8ae3f7 100644 --- a/tests/queries/0_stateless/01045_array_zip.sql +++ b/tests/queries/0_stateless/01045_array_zip.sql @@ -2,7 +2,7 @@ SELECT arrayZip(['a', 'b', 'c'], ['d', 'e', 'f']); SELECT arrayZip(['a', 'b', 'c'], ['d', 'e', 'f'], ['g', 'h', 'i']); -SELECT arrayZip(); -- { serverError 42 } +SELECT arrayZip(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } SELECT arrayZip('a', 'b', 'c'); -- { serverError 43 } diff --git a/tests/queries/0_stateless/02354_parse_timedelta.sql b/tests/queries/0_stateless/02354_parse_timedelta.sql index 29f2bf9fdfce..e876de643747 100644 --- a/tests/queries/0_stateless/02354_parse_timedelta.sql +++ b/tests/queries/0_stateless/02354_parse_timedelta.sql @@ -11,8 +11,8 @@ SELECT parseTimeDelta('1s - 1ms : 1μs ; 1ns'); SELECT parseTimeDelta('1.11s1.11ms1.11us1.11ns'); -- invalid expressions -SELECT parseTimeDelta(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} -SELECT parseTimeDelta('1yr', 1); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +SELECT parseTimeDelta(); -- {serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION} +SELECT parseTimeDelta('1yr', 1); -- {serverError TOO_MANY_ARGUMENTS_FOR_FUNCTION} SELECT parseTimeDelta(1); -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} SELECT parseTimeDelta(' '); -- {serverError BAD_ARGUMENTS} SELECT parseTimeDelta('-1yr'); -- {serverError BAD_ARGUMENTS} diff --git a/tests/queries/0_stateless/02372_now_in_block.sql b/tests/queries/0_stateless/02372_now_in_block.sql index 7c884c0ba7d2..815f74e58452 100644 --- a/tests/queries/0_stateless/02372_now_in_block.sql +++ b/tests/queries/0_stateless/02372_now_in_block.sql @@ -1,4 +1,4 @@ SELECT count() FROM (SELECT DISTINCT nowInBlock(), nowInBlock('Pacific/Pitcairn') FROM system.numbers LIMIT 2); SELECT nowInBlock(1); -- { serverError 43 } SELECT nowInBlock(NULL) IS NULL; -SELECT nowInBlock('UTC', 'UTC'); -- { serverError 42 } +SELECT nowInBlock('UTC', 'UTC'); -- { serverError TOO_MANY_ARGUMENTS_FOR_FUNCTION } diff --git a/tests/queries/0_stateless/02718_array_fold.sql b/tests/queries/0_stateless/02718_array_fold.sql index 7dee33c4705c..73a3cebdda16 100644 --- a/tests/queries/0_stateless/02718_array_fold.sql +++ b/tests/queries/0_stateless/02718_array_fold.sql @@ -1,10 +1,10 @@ SELECT '-- Negative tests'; -SELECT arrayFold(); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT arrayFold(1); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT arrayFold(1, toUInt64(0)); -- { serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT arrayFold(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } +SELECT arrayFold(1); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } +SELECT arrayFold(1, toUInt64(0)); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } SELECT arrayFold(1, emptyArrayUInt64(), toUInt64(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x -> x, emptyArrayString(), toInt8(0)); -- { serverError TYPE_MISMATCH } -SELECT arrayFold( acc,x -> x, 'not an array', toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayFold( acc,x -> x, 'not an array', toUInt8(0)); -- { serverError I02718_array_foldLLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x,y -> x, [0, 1], 'not an array', toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x -> x, [0, 1], [2, 3], toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x,y -> x, [0, 1], [2, 3, 4], toUInt8(0)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } From c45c1444355b740ed01fe1849f18f17c689d826b Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Tue, 7 May 2024 16:39:43 +0200 Subject: [PATCH 14/46] fix typo in fast tests --- tests/queries/0_stateless/02718_array_fold.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02718_array_fold.sql b/tests/queries/0_stateless/02718_array_fold.sql index 73a3cebdda16..e59eae87fdf7 100644 --- a/tests/queries/0_stateless/02718_array_fold.sql +++ b/tests/queries/0_stateless/02718_array_fold.sql @@ -4,7 +4,7 @@ SELECT arrayFold(1); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } SELECT arrayFold(1, toUInt64(0)); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } SELECT arrayFold(1, emptyArrayUInt64(), toUInt64(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x -> x, emptyArrayString(), toInt8(0)); -- { serverError TYPE_MISMATCH } -SELECT arrayFold( acc,x -> x, 'not an array', toUInt8(0)); -- { serverError I02718_array_foldLLEGAL_TYPE_OF_ARGUMENT } +SELECT arrayFold( acc,x -> x, 'not an array', toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x,y -> x, [0, 1], 'not an array', toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x -> x, [0, 1], [2, 3], toUInt8(0)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT arrayFold( acc,x,y -> x, [0, 1], [2, 3, 4], toUInt8(0)); -- { serverError SIZES_OF_ARRAYS_DONT_MATCH } From 5117422c7bebd04e3511abff3b02fd26561cda13 Mon Sep 17 00:00:00 2001 From: yariks5s Date: Tue, 7 May 2024 14:48:50 +0000 Subject: [PATCH 15/46] init --- src/Core/Settings.h | 1 + src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 1 + .../Formats/Impl/PrettyBlockOutputFormat.cpp | 10 +++++----- .../Impl/PrettyCompactBlockOutputFormat.cpp | 4 ++-- .../Impl/PrettySpaceBlockOutputFormat.cpp | 4 ++-- .../03132_pretty_format_break_line.reference | 19 +++++++++++++++++++ .../03132_pretty_format_break_line.sql | 7 +++++++ 8 files changed, 38 insertions(+), 9 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b4313d9af561..d640a556fb62 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1113,6 +1113,7 @@ class IColumn; \ M(String, format_json_object_each_row_column_for_object_name, "", "The name of column that will be used as object names in JSONObjectEachRow format. Column type should be String", 0) \ \ + M(Bool, output_format_pretty_preserve_border_for_multiline_string, true, "Applies better rendering for multiline strings.", 0) \ M(UInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \ M(UInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.", 0) \ M(UInt64, output_format_pretty_max_value_width, 10000, "Maximum width of value to display in Pretty formats. If greater - it will be cut.", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index b7e9899da46d..cca4c30f5e02 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -173,6 +173,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se format_settings.pretty.max_value_width_apply_for_single_value = settings.output_format_pretty_max_value_width_apply_for_single_value; format_settings.pretty.highlight_digit_groups = settings.output_format_pretty_highlight_digit_groups; format_settings.pretty.output_format_pretty_row_numbers = settings.output_format_pretty_row_numbers; + format_settings.pretty.preserve_border_for_multiline_string = settings.output_format_pretty_preserve_border_for_multiline_string; format_settings.pretty.output_format_pretty_single_large_number_tip_threshold = settings.output_format_pretty_single_large_number_tip_threshold; format_settings.protobuf.input_flatten_google_wrappers = settings.input_format_protobuf_flatten_google_wrappers; format_settings.protobuf.output_nullables_with_google_wrappers = settings.output_format_protobuf_nullables_with_google_wrappers; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index da225a39ec9b..e320ea8e6b66 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -282,6 +282,7 @@ struct FormatSettings SettingFieldUInt64Auto color{"auto"}; bool output_format_pretty_row_numbers = false; + bool preserve_border_for_multiline_string = true; UInt64 output_format_pretty_single_large_number_tip_threshold = 1'000'000; enum class Charset diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 178d0b912e1a..e23622bc2e5c 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -38,7 +38,7 @@ void PrettyBlockOutputFormat::calculateWidths( max_padded_widths.resize_fill(num_columns); name_widths.resize(num_columns); - const bool need_cut_to_width = format_settings.pretty.max_value_width_apply_for_single_value || num_rows != 1 || num_columns != 1 || total_rows != 0; + const bool need_cut_to_width = format_settings.pretty.preserve_border_for_multiline_string && (format_settings.pretty.max_value_width_apply_for_single_value || num_rows != 1 || num_columns != 1 || total_rows != 0); /// Calculate widths of all values. String serialized_value; @@ -333,7 +333,7 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); serializations[j]->serializeText(*columns[j], i, out_serialize, format_settings); } - if (cut_to_width) + if (cut_to_width && format_settings.pretty.preserve_border_for_multiline_string) splitValueAtBreakLine(serialized_value, transferred_row[j], cur_width); has_transferred_row |= !transferred_row[j].empty() && cur_width <= cut_to_width; @@ -345,7 +345,7 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind writeReadableNumberTip(chunk); writeCString("\n", out); - if (has_transferred_row) + if (has_transferred_row && format_settings.pretty.preserve_border_for_multiline_string) writeTransferredRow(max_widths, header, transferred_row, cut_to_width, false); } @@ -453,7 +453,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( value_width = format_settings.pretty.max_value_width; has_break_line = false; } - else if (!has_break_line) + else if (!has_break_line || !format_settings.pretty.preserve_border_for_multiline_string) value += ' '; auto write_padding = [&]() @@ -478,7 +478,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( write_padding(); } - if (has_break_line) + if (has_break_line && format_settings.pretty.preserve_border_for_multiline_string) writeString("…", out); } diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 345b6c84ecd5..1ccb6d713d7c 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -182,7 +182,7 @@ void PrettyCompactBlockOutputFormat::writeRow( WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); serializations[j]->serializeText(*columns[j], row_num, out_serialize, format_settings); } - if (cut_to_width) + if (cut_to_width && format_settings.pretty.preserve_border_for_multiline_string) splitValueAtBreakLine(serialized_value, transferred_row[j], cur_width); has_transferred_row |= !transferred_row[j].empty() && cur_width <= cut_to_width; @@ -194,7 +194,7 @@ void PrettyCompactBlockOutputFormat::writeRow( writeReadableNumberTip(chunk); writeCString("\n", out); - if (has_transferred_row) + if (has_transferred_row && format_settings.pretty.preserve_border_for_multiline_string) writeTransferredRow(max_widths, header, transferred_row, cut_to_width, false); } diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 6940c20e25ba..0ed8c4398e4c 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -100,7 +100,7 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port WriteBufferFromString out_serialize(serialized_value, AppendModeTag()); serializations[column]->serializeText(*columns[column], row, out_serialize, format_settings); } - if (cut_to_width) + if (cut_to_width && format_settings.pretty.preserve_border_for_multiline_string) splitValueAtBreakLine(serialized_value, transferred_row[column], cur_width); has_transferred_row |= !transferred_row[column].empty() && cur_width <= cut_to_width; @@ -111,7 +111,7 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port writeReadableNumberTip(chunk); writeChar('\n', out); - if (has_transferred_row) + if (has_transferred_row && format_settings.pretty.preserve_border_for_multiline_string) writeTransferredRow(max_widths, header, transferred_row, cut_to_width, true); } diff --git a/tests/queries/0_stateless/03132_pretty_format_break_line.reference b/tests/queries/0_stateless/03132_pretty_format_break_line.reference index a5282f893274..b7b59df24fce 100644 --- a/tests/queries/0_stateless/03132_pretty_format_break_line.reference +++ b/tests/queries/0_stateless/03132_pretty_format_break_line.reference @@ -86,3 +86,22 @@ 2. │ 1 │ hello world │ hellow …│ │ │ │…мир │ └────┴─────────────┴─────────────┘ +┏━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━━━━┓ +┃ id ┃ value ┃ value1 ┃ +┡━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━━━━┩ +│ 0 │ привет +world │ hello world │ +├────┼─────────────┼─────────────┤ +│ 1 │ hello world │ hellow +мир │ +└────┴─────────────┴─────────────┘ + ┏━━━━┳━━━━━━━━━━━━━┳━━━━━━━━━━━━━┓ + ┃ id ┃ value ┃ value1 ┃ + ┡━━━━╇━━━━━━━━━━━━━╇━━━━━━━━━━━━━┩ +1. │ 0 │ привет +world │ hello world │ + ├────┼─────────────┼─────────────┤ +2. │ 1 │ hello world │ hellow +мир │ + └────┴─────────────┴─────────────┘ + \ No newline at end of file diff --git a/tests/queries/0_stateless/03132_pretty_format_break_line.sql b/tests/queries/0_stateless/03132_pretty_format_break_line.sql index ecf967c18369..5f002d8df67b 100644 --- a/tests/queries/0_stateless/03132_pretty_format_break_line.sql +++ b/tests/queries/0_stateless/03132_pretty_format_break_line.sql @@ -1,5 +1,7 @@ DROP TABLE IF EXISTS t_break_line; +SET output_format_pretty_preserve_border_for_multiline_string=1; + CREATE TABLE t_break_line (id UInt64, value String, value1 String) ENGINE=MergeTree ORDER BY id; INSERT INTO t_break_line VALUES(0, 'hello\nworld', 'hello world'); @@ -30,4 +32,9 @@ INSERT INTO t_break_line VALUES(1, 'hello world', 'hellow\nмир'); SELECT * FROM t_break_line ORDER BY id FORMAT PrettyMonoBlock SETTINGS output_format_pretty_row_numbers = 0; SELECT * FROM t_break_line ORDER BY id FORMAT PrettyMonoBlock; +SET output_format_pretty_preserve_border_for_multiline_string=0; + +SELECT * FROM t_break_line ORDER BY id FORMAT PrettyMonoBlock SETTINGS output_format_pretty_row_numbers = 0; +SELECT * FROM t_break_line ORDER BY id FORMAT PrettyMonoBlock; + DROP TABLE t_break_line; \ No newline at end of file From ac485b612de0e7e79188fdb07cd63a3500f92132 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 7 May 2024 17:11:37 +0200 Subject: [PATCH 16/46] add setting to SettingsChangesHistory --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index cd1cd341c294..9a3ed14dd7ca 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -91,6 +91,7 @@ static std::map sett {"cross_join_min_rows_to_compress", 0, 10000000, "A new setting."}, {"cross_join_min_bytes_to_compress", 0, 1_GiB, "A new setting."}, {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, + {"output_format_pretty_preserve_border_for_multiline_string", 1, 1, "Applies better rendering for multiline strings."}, }}, {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, From 8e072f6b8d567ea82aa0c44bad6f26228b7bb96f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 7 May 2024 18:58:51 +0200 Subject: [PATCH 17/46] Newline at the end of .reference --- .../queries/0_stateless/03132_pretty_format_break_line.reference | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/03132_pretty_format_break_line.reference b/tests/queries/0_stateless/03132_pretty_format_break_line.reference index b7b59df24fce..06b17ce4e12f 100644 --- a/tests/queries/0_stateless/03132_pretty_format_break_line.reference +++ b/tests/queries/0_stateless/03132_pretty_format_break_line.reference @@ -104,4 +104,3 @@ world │ hello world │ 2. │ 1 │ hello world │ hellow мир │ └────┴─────────────┴─────────────┘ - \ No newline at end of file From ed59a5c248e613aeab40f98dfedd2e9cd5c88dee Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 7 May 2024 19:17:00 +0200 Subject: [PATCH 18/46] Update PrettyBlockOutputFormat.cpp --- src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index e23622bc2e5c..d84ffe800e72 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -345,7 +345,7 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind writeReadableNumberTip(chunk); writeCString("\n", out); - if (has_transferred_row && format_settings.pretty.preserve_border_for_multiline_string) + if (has_transferred_row) writeTransferredRow(max_widths, header, transferred_row, cut_to_width, false); } From cdcdb4cf71420aced463b01869da25be8015b75c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 7 May 2024 19:35:09 +0200 Subject: [PATCH 19/46] remove unnecessary checks --- src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 4 ++-- .../Formats/Impl/PrettyCompactBlockOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index d84ffe800e72..41c7bfa316b2 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -453,7 +453,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( value_width = format_settings.pretty.max_value_width; has_break_line = false; } - else if (!has_break_line || !format_settings.pretty.preserve_border_for_multiline_string) + else if (!has_break_line) value += ' '; auto write_padding = [&]() @@ -478,7 +478,7 @@ void PrettyBlockOutputFormat::writeValueWithPadding( write_padding(); } - if (has_break_line && format_settings.pretty.preserve_border_for_multiline_string) + if (has_break_line) writeString("…", out); } diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 1ccb6d713d7c..ce22a3b2864f 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -194,7 +194,7 @@ void PrettyCompactBlockOutputFormat::writeRow( writeReadableNumberTip(chunk); writeCString("\n", out); - if (has_transferred_row && format_settings.pretty.preserve_border_for_multiline_string) + if (has_transferred_row) writeTransferredRow(max_widths, header, transferred_row, cut_to_width, false); } diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 0ed8c4398e4c..d311f005173c 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -111,7 +111,7 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port writeReadableNumberTip(chunk); writeChar('\n', out); - if (has_transferred_row && format_settings.pretty.preserve_border_for_multiline_string) + if (has_transferred_row) writeTransferredRow(max_widths, header, transferred_row, cut_to_width, true); } From a2e9b6f4c61de96a12edee926c757c3d776d01a7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 7 May 2024 17:42:43 +0000 Subject: [PATCH 20/46] Another attempt. --- src/Planner/PlannerExpressionAnalysis.cpp | 49 ++++++++++++++++++++--- 1 file changed, 43 insertions(+), 6 deletions(-) diff --git a/src/Planner/PlannerExpressionAnalysis.cpp b/src/Planner/PlannerExpressionAnalysis.cpp index d7fa270a6430..6e194b2c03ec 100644 --- a/src/Planner/PlannerExpressionAnalysis.cpp +++ b/src/Planner/PlannerExpressionAnalysis.cpp @@ -51,6 +51,41 @@ FilterAnalysisResult analyzeFilter(const QueryTreeNodePtr & filter_expression_no return result; } +bool isDeterministicConstant(const ConstantNode & root) +{ + const auto & source_expression = root.getSourceExpression(); + if (!source_expression) + return true; + + std::stack nodes; + nodes.push(source_expression.get()); + while (!nodes.empty()) + { + const auto * node = nodes.top(); + nodes.pop(); + + const auto * constant_node = node->as(); + const auto * function_node = node->as(); + if (constant_node) + { + if (!isDeterministicConstant(*constant_node)) + return false; + } + else if (function_node) + { + if (!function_node->getFunctionOrThrow()->isDeterministic()) + return false; + + for (const auto & child : function_node->getArguments()) + nodes.push(child.get()); + } + else + return false; + } + + return true; +} + /** Construct aggregation analysis result if query tree has GROUP BY or aggregates. * Actions before aggregation are added into actions chain, if result is not null optional. */ @@ -86,6 +121,8 @@ std::optional analyzeAggregation(const QueryTreeNodeP (query_node.isGroupByWithGroupingSets() || query_node.isGroupByWithRollup() || query_node.isGroupByWithCube()); bool is_secondary_query = planner_context->getQueryContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; + bool is_distributed_query = planner_context->getQueryContext()->isDistributed(); + bool check_deterministic_constants = is_secondary_query || is_distributed_query; if (query_node.hasGroupBy()) { @@ -99,10 +136,10 @@ std::optional analyzeAggregation(const QueryTreeNodeP for (auto & grouping_set_key_node : grouping_set_keys_list_node_typed.getNodes()) { - auto is_constant_key = grouping_set_key_node->as() != nullptr; - group_by_with_constant_keys |= is_constant_key; + const auto * constant_key = grouping_set_key_node->as(); + group_by_with_constant_keys |= (constant_key != nullptr); - if (!is_secondary_query && is_constant_key && !aggregates_descriptions.empty()) + if (constant_key && !aggregates_descriptions.empty() && (!check_deterministic_constants || isDeterministicConstant(*constant_key))) continue; auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, grouping_set_key_node); @@ -151,10 +188,10 @@ std::optional analyzeAggregation(const QueryTreeNodeP { for (auto & group_by_key_node : query_node.getGroupBy().getNodes()) { - auto is_constant_key = group_by_key_node->as() != nullptr; - group_by_with_constant_keys |= is_constant_key; + const auto * constant_key = group_by_key_node->as(); + group_by_with_constant_keys |= (constant_key != nullptr); - if (!is_secondary_query && is_constant_key && !aggregates_descriptions.empty()) + if (constant_key && !aggregates_descriptions.empty() && (!check_deterministic_constants || isDeterministicConstant(*constant_key))) continue; auto expression_dag_nodes = actions_visitor.visit(before_aggregation_actions, group_by_key_node); From f45e032715dcc04d3ef78d84750c0f5f835b5008 Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Tue, 7 May 2024 21:11:04 +0200 Subject: [PATCH 21/46] fix errorCodes in stateless tests --- tests/queries/0_stateless/00727_concat.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00727_concat.sql b/tests/queries/0_stateless/00727_concat.sql index f5048dcaaae9..01792545b5a4 100644 --- a/tests/queries/0_stateless/00727_concat.sql +++ b/tests/queries/0_stateless/00727_concat.sql @@ -93,4 +93,4 @@ SELECT concat(materialize(NULL :: Nullable(UInt64))); SELECT CONCAT('Testing the ', 'alias'); -SELECT concat(); -- { serverError 42 } +SELECT concat(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION } From f4e26052086fc370ad65f729099568e7d8549247 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 May 2024 04:10:01 +0200 Subject: [PATCH 22/46] Insert symbols info to the CI database --- tests/ci/ci.py | 37 ++++++++++++++++++- .../prepare-time-trace/prepare-time-trace.sh | 32 ++++++++++++++++ 2 files changed, 67 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index d6b1866c563d..1229ec33a25f 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1745,7 +1745,7 @@ def _upload_build_profile_data( profile_data_file = Path(TEMP_PATH) / "profile.json" with open(profile_data_file, "wb") as profile_fd: for profile_source in profiles_dir.iterdir(): - if profile_source.name != "binary_sizes.txt": + if profile_source.name != "binary_sizes.txt" and profile_source.name != "binary_symbols.txt": with open(profiles_dir / profile_source, "rb") as ps_fd: profile_fd.write(ps_fd.read()) @@ -1787,7 +1787,40 @@ def _upload_build_profile_data( try: ch_helper.insert_file(url, auth, query, binary_sizes_file) except InsertException: - logging.error("Failed to insert binary_size_file for the build, continue") + logging.error("Failed to insert binary_sizes_file for the build, continue") + + query = f"""INSERT INTO binary_symbols + ( + pull_request_number, + commit_sha, + check_start_time, + check_name, + instance_type, + instance_id, + file, + address, + size, + type, + symbol, + ) + SELECT {pr_info.number}, '{pr_info.sha}', '{job_report.start_time}', '{build_name}', '{instance_type}', '{instance_id}', + file, reinterpretAsUInt64(reverse(unhex(address))), reinterpretAsUInt64(reverse(unhex(size))), type, symbol + FROM input('file String, address String, size String, type String, symbol String') + SETTINGS format_regexp = '^([^ ]+) ([0-9a-fA-F]+) ([0-9a-fA-F]+) (.) (.+)$' + FORMAT Regexp""" + + binary_symbols_file = profiles_dir / "binary_symbols.txt" + + print( + "::notice ::Log Uploading binary symbols data, path: %s, size: %s, query: %s", + binary_symbols_file, + binary_symbols_file.stat().st_size, + query, + ) + try: + ch_helper.insert_file(url, auth, query, binary_symbols_file) + except InsertException: + logging.error("Failed to insert binary_symbols_file for the build, continue") def _add_build_to_version_history( diff --git a/utils/prepare-time-trace/prepare-time-trace.sh b/utils/prepare-time-trace/prepare-time-trace.sh index 812928e8bd8c..2ecc1ddcd64f 100755 --- a/utils/prepare-time-trace/prepare-time-trace.sh +++ b/utils/prepare-time-trace/prepare-time-trace.sh @@ -82,3 +82,35 @@ ORDER BY (date, file, pull_request_number, commit_sha, check_name); /// find "$INPUT_DIR" -type f -executable -or -name '*.o' -or -name '*.a' | grep -v cargo | xargs wc -c | grep -v 'total' > "${OUTPUT_DIR}/binary_sizes.txt" + +# Additionally, collect information about the symbols inside translation units +true< '{}.symbols' +" + +find "$INPUT_DIR" -type f -name '*.o.symbols' | xargs cat > "${OUTPUT_DIR}/binary_symbols.txt" From 9c449cc4535fa7147a65ca9ff4a2eb4a37620323 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 May 2024 04:18:01 +0200 Subject: [PATCH 23/46] Insert symbols info to the CI database --- tests/ci/ci.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 1229ec33a25f..ddad84d3e3eb 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1806,7 +1806,7 @@ def _upload_build_profile_data( SELECT {pr_info.number}, '{pr_info.sha}', '{job_report.start_time}', '{build_name}', '{instance_type}', '{instance_id}', file, reinterpretAsUInt64(reverse(unhex(address))), reinterpretAsUInt64(reverse(unhex(size))), type, symbol FROM input('file String, address String, size String, type String, symbol String') - SETTINGS format_regexp = '^([^ ]+) ([0-9a-fA-F]+) ([0-9a-fA-F]+) (.) (.+)$' + SETTINGS format_regexp = '^([^ ]+) ([0-9a-fA-F]+)(?: ([0-9a-fA-F]+))? (.) (.+)$' FORMAT Regexp""" binary_symbols_file = profiles_dir / "binary_symbols.txt" From dc0df8cfec4d9c616a8c4f54489f57136691dbe9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 8 May 2024 02:31:32 +0000 Subject: [PATCH 24/46] Automatic style fix --- tests/ci/ci.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index ddad84d3e3eb..7779f8d73498 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1745,7 +1745,10 @@ def _upload_build_profile_data( profile_data_file = Path(TEMP_PATH) / "profile.json" with open(profile_data_file, "wb") as profile_fd: for profile_source in profiles_dir.iterdir(): - if profile_source.name != "binary_sizes.txt" and profile_source.name != "binary_symbols.txt": + if ( + profile_source.name != "binary_sizes.txt" + and profile_source.name != "binary_symbols.txt" + ): with open(profiles_dir / profile_source, "rb") as ps_fd: profile_fd.write(ps_fd.read()) @@ -1820,7 +1823,9 @@ def _upload_build_profile_data( try: ch_helper.insert_file(url, auth, query, binary_symbols_file) except InsertException: - logging.error("Failed to insert binary_symbols_file for the build, continue") + logging.error( + "Failed to insert binary_symbols_file for the build, continue" + ) def _add_build_to_version_history( From f9da558a2c9f37ae4967f352d18f7b9f1fe91e65 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 May 2024 05:20:07 +0200 Subject: [PATCH 25/46] Your lint --- tests/ci/ci.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 7779f8d73498..11803a5bc263 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1745,10 +1745,7 @@ def _upload_build_profile_data( profile_data_file = Path(TEMP_PATH) / "profile.json" with open(profile_data_file, "wb") as profile_fd: for profile_source in profiles_dir.iterdir(): - if ( - profile_source.name != "binary_sizes.txt" - and profile_source.name != "binary_symbols.txt" - ): + if profile_source.name not in ('binary_sizes.txt', 'binary_symbols.txt'): with open(profiles_dir / profile_source, "rb") as ps_fd: profile_fd.write(ps_fd.read()) From 71eab76f5cd8574d302a2edb81f1efaa57230919 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 8 May 2024 03:31:41 +0000 Subject: [PATCH 26/46] Automatic style fix --- tests/ci/ci.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 11803a5bc263..5f42309d4396 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -1745,7 +1745,10 @@ def _upload_build_profile_data( profile_data_file = Path(TEMP_PATH) / "profile.json" with open(profile_data_file, "wb") as profile_fd: for profile_source in profiles_dir.iterdir(): - if profile_source.name not in ('binary_sizes.txt', 'binary_symbols.txt'): + if profile_source.name not in ( + "binary_sizes.txt", + "binary_symbols.txt", + ): with open(profiles_dir / profile_source, "rb") as ps_fd: profile_fd.write(ps_fd.read()) From c3d01a91d544b639eafd4991096c5b8757441d1f Mon Sep 17 00:00:00 2001 From: Yohann Jardin Date: Wed, 8 May 2024 09:31:06 +0200 Subject: [PATCH 27/46] empty commit From b57ad0f0eb1063cd3e525c76514ed2159578810d Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 8 May 2024 09:58:01 +0200 Subject: [PATCH 28/46] Analyzer fix 03024_total_rows_approx_is_set_for_system_zeros_and_generate_random --- src/Planner/PlannerJoinTree.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 514c19b0f89c..bf1ebe6740b7 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -691,6 +691,9 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres { if (max_block_size_limited < select_query_info.local_storage_limits.local_limits.size_limits.max_rows) table_expression_query_info.limit = max_block_size_limited; + /// Ask to read just enough rows to make the max_rows limit effective (so it has a chance to be triggered). + else if (select_query_info.local_storage_limits.local_limits.size_limits.max_rows < std::numeric_limits::max()) + table_expression_query_info.limit = 1 + select_query_info.local_storage_limits.local_limits.size_limits.max_rows; } else { From 0efc7c73cfd4c5c2eefe8256d7fe41c2c2a26c84 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 8 May 2024 14:28:06 +0200 Subject: [PATCH 29/46] MergeTreePrefetchedReadPool: Print parent name when logging projection parts --- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index 6d2875b8d9f0..80deed27b8fa 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -384,6 +384,15 @@ void MergeTreePrefetchedReadPool::fillPerPartStatistics() } } +namespace +{ +ALWAYS_INLINE inline String getPartNameForLogging(const DataPartPtr & part) +{ + return part->isProjectionPart() ? fmt::format("{}.{}", part->name, part->getParentPartName()) : part->name; +} +} + + void MergeTreePrefetchedReadPool::fillPerThreadTasks(size_t threads, size_t sum_marks) { if (per_part_infos.empty()) @@ -438,7 +447,7 @@ void MergeTreePrefetchedReadPool::fillPerThreadTasks(size_t threads, size_t sum_ LOG_DEBUG( log, "Part: {}, sum_marks: {}, approx mark size: {}, prefetch_step_bytes: {}, prefetch_step_marks: {}, (ranges: {})", - parts_ranges[i].data_part->name, + getPartNameForLogging(parts_ranges[i].data_part), part_stat.sum_marks, part_stat.approx_size_of_mark, settings.filesystem_prefetch_step_bytes, @@ -522,7 +531,9 @@ void MergeTreePrefetchedReadPool::fillPerThreadTasks(size_t threads, size_t sum_ throw Exception( ErrorCodes::LOGICAL_ERROR, "Requested {} marks from part {}, but part has only {} marks", - marks_to_get_from_part, per_part_infos[part_idx]->data_part->name, part_stat.sum_marks); + marks_to_get_from_part, + getPartNameForLogging(per_part_infos[part_idx]->data_part), + part_stat.sum_marks); } size_t num_marks_to_get = marks_to_get_from_part; @@ -598,7 +609,7 @@ std::string MergeTreePrefetchedReadPool::dumpTasks(const TasksPerThread & tasks) result << '\t'; result << ++no << ": "; result << "reader future: " << task->isValidReadersFuture() << ", "; - result << "part: " << task->read_info->data_part->name << ", "; + result << "part: " << getPartNameForLogging(task->read_info->data_part) << ", "; result << "ranges: " << toString(task->ranges); } } From 16eb12a321c17259f368d1dc3588a6bdc7de814f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 8 May 2024 14:58:21 +0200 Subject: [PATCH 30/46] Correctly stop copy tasks when exception thrown --- src/Disks/IDisk.cpp | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 14d5f94ef469..0f17aabd2106 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -151,17 +151,19 @@ void IDisk::copyThroughBuffers( { ResultsCollector results; + SCOPE_EXIT_SAFE( + for (auto & result : results) + result.wait(); + for (auto & result : results) + result.get(); /// May rethrow an exception + ); + /// Disable parallel write. We already copy in parallel. /// Avoid high memory usage. See test_s3_zero_copy_ttl/test.py::test_move_and_s3_memory_usage write_settings.s3_allow_parallel_part_upload = false; write_settings.azure_allow_parallel_part_upload = false; asyncCopy(*this, from_path, *to_disk, to_path, copying_thread_pool, results, copy_root_dir, read_settings, write_settings, cancellation_hook); - - for (auto & result : results) - result.wait(); - for (auto & result : results) - result.get(); /// May rethrow an exception } From 3d5c8db1e07eeba4d5cd13c501bc1b937c058575 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 8 May 2024 15:05:06 +0200 Subject: [PATCH 31/46] Use ThreadPoolCallbackRunnerLocal --- src/Disks/IDisk.cpp | 48 +++++++++++---------------------------------- 1 file changed, 11 insertions(+), 37 deletions(-) diff --git a/src/Disks/IDisk.cpp b/src/Disks/IDisk.cpp index 0f17aabd2106..5f0ca850b407 100644 --- a/src/Disks/IDisk.cpp +++ b/src/Disks/IDisk.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -84,16 +85,12 @@ UInt128 IDisk::getEncryptedFileIV(const String &) const throw Exception(ErrorCodes::NOT_IMPLEMENTED, "File encryption is not implemented for disk of type {}", getDataSourceDescription().type); } - -using ResultsCollector = std::vector>; - void asyncCopy( IDisk & from_disk, String from_path, IDisk & to_disk, String to_path, - ThreadPool & pool, - ResultsCollector & results, + ThreadPoolCallbackRunnerLocal & runner, bool copy_root_dir, const ReadSettings & read_settings, const WriteSettings & write_settings, @@ -101,29 +98,11 @@ void asyncCopy( { if (from_disk.isFile(from_path)) { - auto promise = std::make_shared>(); - auto future = promise->get_future(); - - pool.scheduleOrThrowOnError( - [&from_disk, from_path, &to_disk, to_path, &read_settings, &write_settings, promise, thread_group = CurrentThread::getGroup(), &cancellation_hook]() - { - try - { - SCOPE_EXIT_SAFE(if (thread_group) CurrentThread::detachFromGroupIfNotDetached();); - - if (thread_group) - CurrentThread::attachToGroup(thread_group); - - from_disk.copyFile(from_path, to_disk, fs::path(to_path) / fileName(from_path), read_settings, write_settings, cancellation_hook); - promise->set_value(); - } - catch (...) - { - promise->set_exception(std::current_exception()); - } + runner( + [&from_disk, from_path, &to_disk, to_path, &read_settings, &write_settings, &cancellation_hook] { + from_disk.copyFile( + from_path, to_disk, fs::path(to_path) / fileName(from_path), read_settings, write_settings, cancellation_hook); }); - - results.push_back(std::move(future)); } else { @@ -136,7 +115,7 @@ void asyncCopy( } for (auto it = from_disk.iterateDirectory(from_path); it->isValid(); it->next()) - asyncCopy(from_disk, it->path(), to_disk, dest, pool, results, true, read_settings, write_settings, cancellation_hook); + asyncCopy(from_disk, it->path(), to_disk, dest, runner, true, read_settings, write_settings, cancellation_hook); } } @@ -149,21 +128,16 @@ void IDisk::copyThroughBuffers( WriteSettings write_settings, const std::function & cancellation_hook) { - ResultsCollector results; - - SCOPE_EXIT_SAFE( - for (auto & result : results) - result.wait(); - for (auto & result : results) - result.get(); /// May rethrow an exception - ); + ThreadPoolCallbackRunnerLocal runner(copying_thread_pool, "AsyncCopy"); /// Disable parallel write. We already copy in parallel. /// Avoid high memory usage. See test_s3_zero_copy_ttl/test.py::test_move_and_s3_memory_usage write_settings.s3_allow_parallel_part_upload = false; write_settings.azure_allow_parallel_part_upload = false; - asyncCopy(*this, from_path, *to_disk, to_path, copying_thread_pool, results, copy_root_dir, read_settings, write_settings, cancellation_hook); + asyncCopy(*this, from_path, *to_disk, to_path, runner, copy_root_dir, read_settings, write_settings, cancellation_hook); + + runner.waitForAllToFinishAndRethrowFirstError(); } From 1e036498260968f323bb50eabe0a0048416de334 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 8 May 2024 16:14:15 +0200 Subject: [PATCH 32/46] Revert "Fix index analysis for `DateTime64`" --- src/Functions/FunctionsConversion.cpp | 2 +- .../02373_datetime64_monotonicity.queries | 57 +++++++++++++++++++ .../02373_datetime64_monotonicity.reference | 12 ---- .../02373_datetime64_monotonicity.sh | 40 +------------ ...tetime64_constant_index_analysis.reference | 6 -- ...147_datetime64_constant_index_analysis.sql | 12 ---- 6 files changed, 59 insertions(+), 70 deletions(-) create mode 100644 tests/queries/0_stateless/02373_datetime64_monotonicity.queries delete mode 100644 tests/queries/0_stateless/03147_datetime64_constant_index_analysis.reference delete mode 100644 tests/queries/0_stateless/03147_datetime64_constant_index_analysis.sql diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index 0da84d6d224c..a16ce136b9a7 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4853,7 +4853,7 @@ FunctionBasePtr createFunctionBaseCast( DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64, DataTypeUInt128, DataTypeUInt256, DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64, DataTypeInt128, DataTypeInt256, DataTypeFloat32, DataTypeFloat64, - DataTypeDate, DataTypeDate32, DataTypeDateTime, DataTypeDateTime64, + DataTypeDate, DataTypeDate32, DataTypeDateTime, DataTypeString>(return_type.get(), [&](auto & type) { monotonicity = FunctionTo>::Type::Monotonic::get; diff --git a/tests/queries/0_stateless/02373_datetime64_monotonicity.queries b/tests/queries/0_stateless/02373_datetime64_monotonicity.queries new file mode 100644 index 000000000000..404e3391205b --- /dev/null +++ b/tests/queries/0_stateless/02373_datetime64_monotonicity.queries @@ -0,0 +1,57 @@ +drop table if exists dt64_monot_test; +drop table if exists dt64_monot_test_string; +CREATE TABLE dt64_monot_test(`date_time` DateTime64(3, 'Europe/Berlin'), `id` String) ENGINE = MergeTree PARTITION BY toDate(date_time, 'Europe/Berlin') ORDER BY date_time; +insert into dt64_monot_test select toDateTime64('2020-01-01 00:00:00.000',3)+number , '' from numbers(10); + +SELECT count() FROM dt64_monot_test WHERE toDateTime(date_time) >= toDateTime('2020-01-01 00:00:00') SETTINGS force_index_by_date = 1, force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time, 3) >= '2020-01-01 00:00:01.111' SETTINGS force_index_by_date = 1, force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time, 3) >= '2020-01-01 00:00:00.000' SETTINGS force_index_by_date = 1, force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001', 3) SETTINGS force_index_by_date = 1, force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001', 3, 'Europe/Berlin') SETTINGS force_index_by_date = 1, force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001',6) SETTINGS force_index_by_date = 1; -- { serverError 277} + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') SETTINGS force_primary_key = 1; -- { serverError 277} + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001',6) SETTINGS force_primary_key = 1; -- { serverError 277} + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) <= toDateTime64('2020-01-01 00:00:00.000001',3, 'Europe/Berlin') settings force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) <= toDateTime64('2020-01-01 00:00:00.000001',3) settings force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) = toDateTime64('2020-01-01 00:00:00.000000',6); + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) = toDateTime64('2020-01-01 00:00:00.000000',6, 'Europe/Berlin'); + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) = toDateTime64('2020-01-01 00:00:00.000000',6) settings force_index_by_date = 1, force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) = toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) > toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) >= toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) >= toDateTime64('2020-01-01 00:00:00.000001',6) settings force_index_by_date = 1, force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= toDateTime64('2020-01-01 00:00:00.000001',0, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= toDateTime64('2020-01-01 00:00:00.000001',0) settings force_index_by_date = 1, force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= '2020-01-01 00:00:00' settings force_index_by_date = 1, force_primary_key = 1; + +SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= '2020-01-01 00:00:01.1' settings force_index_by_date = 1, force_primary_key = 1; + +create table dt64_monot_test_string(date_time String, x String) Engine=MergeTree order by date_time; +insert into dt64_monot_test_string select '2020-01-01 00:00:00.000000001', '' from numbers(1); +insert into dt64_monot_test_string select '2020-01-01 00:00:00.000', '' from numbers(10); + +SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00.000000000'; +SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,3) = '2020-01-01 00:00:00.000000001'; +SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00'; + +drop table dt64_monot_test; +drop table dt64_monot_test_string; diff --git a/tests/queries/0_stateless/02373_datetime64_monotonicity.reference b/tests/queries/0_stateless/02373_datetime64_monotonicity.reference index dd7ddf43e704..935ee685cc99 100644 --- a/tests/queries/0_stateless/02373_datetime64_monotonicity.reference +++ b/tests/queries/0_stateless/02373_datetime64_monotonicity.reference @@ -4,9 +4,6 @@ Asia/Tehran 0 10 0 -9 -0 -9 10 1 1 @@ -30,9 +27,6 @@ UTC 10 10 10 -9 -10 -9 0 1 1 @@ -56,9 +50,6 @@ Canada/Atlantic 10 10 10 -9 -10 -9 0 1 1 @@ -82,9 +73,6 @@ Europe/Berlin 10 10 10 -9 -9 -9 1 1 1 diff --git a/tests/queries/0_stateless/02373_datetime64_monotonicity.sh b/tests/queries/0_stateless/02373_datetime64_monotonicity.sh index 7e852aa24b0b..0e0dc0ec22ab 100755 --- a/tests/queries/0_stateless/02373_datetime64_monotonicity.sh +++ b/tests/queries/0_stateless/02373_datetime64_monotonicity.sh @@ -7,44 +7,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) for tz in Asia/Tehran UTC Canada/Atlantic Europe/Berlin do echo "$tz" - TZ=$tz $CLICKHOUSE_LOCAL --multiline --multiquery " - drop table if exists dt64_monotonicity_test; - drop table if exists dt64_monotonicity_test_string; - CREATE TABLE dt64_monotonicity_test (date_time DateTime64(3, 'Europe/Berlin'), id String) ENGINE = MergeTree PARTITION BY toDate(date_time, 'Europe/Berlin') ORDER BY date_time; - insert into dt64_monotonicity_test select toDateTime64('2020-01-01 00:00:00.000', 3) + number, '' from numbers(10); - - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime(date_time) >= toDateTime('2020-01-01 00:00:00') SETTINGS force_index_by_date = 1, force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= '2020-01-01 00:00:01.111' SETTINGS force_index_by_date = 1, force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= '2020-01-01 00:00:00.000' SETTINGS force_index_by_date = 1, force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 3) SETTINGS force_index_by_date = 1, force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 3, 'Europe/Berlin') SETTINGS force_index_by_date = 1, force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 6) SETTINGS force_index_by_date = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 6, 'Europe/Berlin') SETTINGS force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 6) SETTINGS force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) <= toDateTime64('2020-01-01 00:00:00.000001', 3, 'Europe/Berlin') settings force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) <= toDateTime64('2020-01-01 00:00:00.000001', 3) settings force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) = toDateTime64('2020-01-01 00:00:00.000000', 6); - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) = toDateTime64('2020-01-01 00:00:00.000000', 6, 'Europe/Berlin'); - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) = toDateTime64('2020-01-01 00:00:00.000000', 6) settings force_index_by_date = 1, force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) = toDateTime64('2020-01-01 00:00:00.000001', 6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) > toDateTime64('2020-01-01 00:00:00.000001', 6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) >= toDateTime64('2020-01-01 00:00:00.000001', 6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) >= toDateTime64('2020-01-01 00:00:00.000001', 6) settings force_index_by_date = 1, force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 0) >= toDateTime64('2020-01-01 00:00:00.000001', 0, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 0) >= toDateTime64('2020-01-01 00:00:00.000001', 0) settings force_index_by_date = 1, force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 0) >= '2020-01-01 00:00:00' settings force_index_by_date = 1, force_primary_key = 1; - SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 0) >= '2020-01-01 00:00:01.1' settings force_index_by_date = 1, force_primary_key = 1; - - create table dt64_monotonicity_test_string(date_time String, x String) Engine=MergeTree order by date_time; - insert into dt64_monotonicity_test_string select '2020-01-01 00:00:00.000000001', '' from numbers(1); - insert into dt64_monotonicity_test_string select '2020-01-01 00:00:00.000', '' from numbers(10); - - SELECT count() FROM dt64_monotonicity_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00.000000000'; - SELECT count() FROM dt64_monotonicity_test_string WHERE toDateTime64(date_time,3) = '2020-01-01 00:00:00.000000001'; - SELECT count() FROM dt64_monotonicity_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00'; - - drop table dt64_monotonicity_test; - drop table dt64_monotonicity_test_string; - " + TZ=$tz $CLICKHOUSE_LOCAL -mn < ${CUR_DIR}/02373_datetime64_monotonicity.queries echo "" done diff --git a/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.reference b/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.reference deleted file mode 100644 index 9abcce111362..000000000000 --- a/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.reference +++ /dev/null @@ -1,6 +0,0 @@ -7385 -7385 -7385 -7385 -86401 -86401 diff --git a/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.sql b/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.sql deleted file mode 100644 index 144478eb7215..000000000000 --- a/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.sql +++ /dev/null @@ -1,12 +0,0 @@ -DROP TABLE IF EXISTS test; -CREATE TABLE test (d DateTime, PRIMARY KEY (d)); -INSERT INTO test SELECT toDateTime('2024-01-01') + number FROM numbers(1e6); -SET max_rows_to_read = 10000; -SELECT count() FROM test WHERE d <= '2024-01-01 02:03:04'; -SELECT count() FROM test WHERE d <= toDateTime('2024-01-01 02:03:04'); -SELECT count() FROM test WHERE d <= toDateTime64('2024-01-01 02:03:04', 0); -SELECT count() FROM test WHERE d <= toDateTime64('2024-01-01 02:03:04', 3); -SET max_rows_to_read = 100_000; -SELECT count() FROM test WHERE d <= '2024-01-02'; -SELECT count() FROM test WHERE d <= toDate('2024-01-02'); -DROP TABLE test; From f188866f958094799b8a61601404290672d0df7d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 May 2024 16:17:36 +0200 Subject: [PATCH 33/46] Part of #58061 --- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 2 +- tests/queries/0_stateless/01473_event_time_microseconds.sql | 2 +- .../0_stateless/02226_filesystem_cache_profile_events.sh | 2 +- .../0_stateless/02241_filesystem_cache_on_write_operations.sh | 2 +- tests/queries/0_stateless/02244_hdfs_cluster.sql | 2 +- tests/queries/0_stateless/02458_empty_hdfs_url.sql | 4 ++-- .../0_stateless/02458_hdfs_cluster_schema_inference.sql | 3 +-- .../02536_hdfs_cluster_use_structure_from_table.sql | 3 +-- tests/queries/0_stateless/02723_jit_aggregation_bug_48120.sql | 2 +- .../queries/0_stateless/02919_skip_lots_of_parsing_errors.sh | 3 +-- 10 files changed, 11 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index e36459b019f7..e492ca0aec2e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -28,7 +28,7 @@ namespace ErrorCodes template AnnoyIndexWithSerialization::AnnoyIndexWithSerialization(size_t dimensions) - : Base::AnnoyIndex(dimensions) + : Base::AnnoyIndex(static_cast(dimensions)) { } diff --git a/tests/queries/0_stateless/01473_event_time_microseconds.sql b/tests/queries/0_stateless/01473_event_time_microseconds.sql index 2b443cf82eba..7803c1e2e307 100644 --- a/tests/queries/0_stateless/01473_event_time_microseconds.sql +++ b/tests/queries/0_stateless/01473_event_time_microseconds.sql @@ -1,4 +1,4 @@ --- Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug, no-cpu-aarch64 +-- Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug -- This file contains tests for the event_time_microseconds field for various tables. -- Note: Only event_time_microseconds for asynchronous_metric_log table is tested via diff --git a/tests/queries/0_stateless/02226_filesystem_cache_profile_events.sh b/tests/queries/0_stateless/02226_filesystem_cache_profile_events.sh index 02e98bbb1b0e..9d87542d84de 100755 --- a/tests/queries/0_stateless/02226_filesystem_cache_profile_events.sh +++ b/tests/queries/0_stateless/02226_filesystem_cache_profile_events.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-parallel, no-s3-storage, no-random-settings, no-cpu-aarch64, no-replicated-database +# Tags: no-fasttest, no-parallel, no-s3-storage, no-random-settings, no-replicated-database # set -x diff --git a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.sh b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.sh index ee1d942a4216..c1d930f54a75 100755 --- a/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.sh +++ b/tests/queries/0_stateless/02241_filesystem_cache_on_write_operations.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long, no-fasttest, no-parallel, no-s3-storage, no-random-settings, no-cpu-aarch64 +# Tags: long, no-fasttest, no-parallel, no-s3-storage, no-random-settings # set -x diff --git a/tests/queries/0_stateless/02244_hdfs_cluster.sql b/tests/queries/0_stateless/02244_hdfs_cluster.sql index ffd4a35a5062..fcd47d316be1 100644 --- a/tests/queries/0_stateless/02244_hdfs_cluster.sql +++ b/tests/queries/0_stateless/02244_hdfs_cluster.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-parallel, no-cpu-aarch64 +-- Tags: no-fasttest, no-parallel -- Tag no-fasttest: Depends on Java insert into table function hdfs('hdfs://localhost:12222/test_1.tsv', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32') select 1, 2, 3 settings hdfs_truncate_on_insert=1; diff --git a/tests/queries/0_stateless/02458_empty_hdfs_url.sql b/tests/queries/0_stateless/02458_empty_hdfs_url.sql index ccc554fc6283..9b51740b63df 100644 --- a/tests/queries/0_stateless/02458_empty_hdfs_url.sql +++ b/tests/queries/0_stateless/02458_empty_hdfs_url.sql @@ -1,5 +1,5 @@ --- Tags: no-fasttest, no-cpu-aarch64 +-- Tags: no-fasttest SELECT * FROM hdfsCluster('test_shard_localhost', '', 'TSV'); -- { serverError BAD_ARGUMENTS } SELECT * FROM hdfsCluster('test_shard_localhost', ' ', 'TSV'); -- { serverError BAD_ARGUMENTS } SELECT * FROM hdfsCluster('test_shard_localhost', '/', 'TSV'); -- { serverError BAD_ARGUMENTS } -SELECT * FROM hdfsCluster('test_shard_localhost', 'http/', 'TSV'); -- { serverError BAD_ARGUMENTS } \ No newline at end of file +SELECT * FROM hdfsCluster('test_shard_localhost', 'http/', 'TSV'); -- { serverError BAD_ARGUMENTS } diff --git a/tests/queries/0_stateless/02458_hdfs_cluster_schema_inference.sql b/tests/queries/0_stateless/02458_hdfs_cluster_schema_inference.sql index 42e88fc44b2f..deac4165e94e 100644 --- a/tests/queries/0_stateless/02458_hdfs_cluster_schema_inference.sql +++ b/tests/queries/0_stateless/02458_hdfs_cluster_schema_inference.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-parallel, no-cpu-aarch64 +-- Tags: no-fasttest, no-parallel -- Tag no-fasttest: Depends on Java insert into table function hdfs('hdfs://localhost:12222/test_02458_1.tsv', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32') select 1, 2, 3 settings hdfs_truncate_on_insert=1; @@ -9,4 +9,3 @@ desc hdfsCluster('test_cluster_one_shard_three_replicas_localhost', 'hdfs://loca select * from hdfsCluster('test_cluster_one_shard_three_replicas_localhost', 'hdfs://localhost:12222/test_02458_{1,2}.tsv') order by c1, c2, c3; select * from hdfsCluster('test_cluster_one_shard_three_replicas_localhost', 'hdfs://localhost:12222/test_02458_{1,2}.tsv', 'TSV') order by c1, c2, c3; - diff --git a/tests/queries/0_stateless/02536_hdfs_cluster_use_structure_from_table.sql b/tests/queries/0_stateless/02536_hdfs_cluster_use_structure_from_table.sql index 5d624efc5a95..a395e451f393 100644 --- a/tests/queries/0_stateless/02536_hdfs_cluster_use_structure_from_table.sql +++ b/tests/queries/0_stateless/02536_hdfs_cluster_use_structure_from_table.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-parallel, no-cpu-aarch64 +-- Tags: no-fasttest, no-parallel -- Tag no-fasttest: Depends on Java insert into table function hdfs('hdfs://localhost:12222/test_02536.jsonl', 'TSV') select '{"x" : {"a" : 1, "b" : 2}}' settings hdfs_truncate_on_insert=1; @@ -9,4 +9,3 @@ insert into test select * from hdfsCluster('test_cluster_two_shards_localhost', insert into test select * from hdfsCluster('test_cluster_two_shards_localhost', 'hdfs://localhost:12222/test_02536.jsonl') settings use_structure_from_insertion_table_in_table_functions=1; select * from test; drop table test; - diff --git a/tests/queries/0_stateless/02723_jit_aggregation_bug_48120.sql b/tests/queries/0_stateless/02723_jit_aggregation_bug_48120.sql index 88561f9d895d..1c714e341741 100644 --- a/tests/queries/0_stateless/02723_jit_aggregation_bug_48120.sql +++ b/tests/queries/0_stateless/02723_jit_aggregation_bug_48120.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest, no-cpu-aarch64, no-msan +-- Tags: no-fasttest, no-msan drop table if exists dummy; CREATE TABLE dummy ( num1 Int32, num2 Enum8('foo' = 0, 'bar' = 1, 'tar' = 2) ) diff --git a/tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.sh b/tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.sh index 7ddb55fb39be..64fef943958f 100755 --- a/tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.sh +++ b/tests/queries/0_stateless/02919_skip_lots_of_parsing_errors.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-fasttest, no-cpu-aarch64 +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -20,4 +20,3 @@ $CLICKHOUSE_LOCAL -q "select count() from file('$ERRORS_FILE', CSV)" rm $ERRORS_FILE rm $FILE - From 5885f4263b1ce20db4d5cfa5ed61ec427852471c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 8 May 2024 17:20:38 +0300 Subject: [PATCH 34/46] Recursive CTE documentation fix --- docs/en/sql-reference/statements/select/with.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/statements/select/with.md b/docs/en/sql-reference/statements/select/with.md index ffde7a3fe542..aa0e0c61c4e7 100644 --- a/docs/en/sql-reference/statements/select/with.md +++ b/docs/en/sql-reference/statements/select/with.md @@ -88,7 +88,7 @@ WITH test1 AS (SELECT i + 1, j + 1 FROM test1) SELECT * FROM test1; ``` -# Recursive Queries +## Recursive Queries The optional RECURSIVE modifier allows for a WITH query to refer to its own output. Example: @@ -159,7 +159,7 @@ SELECT * FROM search_tree; └────┴───────────┴───────────┘ ``` -## Search order +### Search order To create a depth-first order, we compute for each result row an array of rows that we have already visited: @@ -211,7 +211,7 @@ SELECT * FROM search_tree ORDER BY depth; └────┴──────┴───────────┴─────────┴───────┘ ``` -## Cycle detection +### Cycle detection First let's create graph table: @@ -291,7 +291,7 @@ SELECT * FROM search_graph WHERE is_cycle ORDER BY from; └──────┴────┴────────┴──────────┴───────────────────────────┘ ``` -## Infinite queries +### Infinite queries It is also possible to use infinite recursive CTE queries if `LIMIT` is used in outer query: From a08a07723bbac42e980e5ff836958e070c69cf4c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 8 May 2024 16:26:03 +0200 Subject: [PATCH 35/46] Update clickhouse-test --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 35568ace72f0..97459bd5b697 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -908,7 +908,7 @@ class MergeTreeSettingsRandomizer: ), "cache_populated_by_fetch": lambda: random.randint(0, 1), "concurrent_part_removal_threshold": threshold_generator(0.2, 0.3, 0, 100), - "old_parts_lifetime": threshold_generator(0.2, 0.3, 30, 8 * 60), + "old_parts_lifetime": threshold_generator(0.2, 0.3, 10, 8 * 60), } @staticmethod From caa150510426e1d4b3a6af18f0107175b24989da Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 8 May 2024 16:43:05 +0200 Subject: [PATCH 36/46] Update 02240_system_filesystem_cache_table.sh --- tests/queries/0_stateless/02240_system_filesystem_cache_table.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02240_system_filesystem_cache_table.sh b/tests/queries/0_stateless/02240_system_filesystem_cache_table.sh index 6a94cffea5ab..9aa631c5d0ae 100755 --- a/tests/queries/0_stateless/02240_system_filesystem_cache_table.sh +++ b/tests/queries/0_stateless/02240_system_filesystem_cache_table.sh @@ -9,7 +9,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) for STORAGE_POLICY in 's3_cache' 'local_cache'; do echo "Using storage policy: $STORAGE_POLICY" - ${CLICKHOUSE_CLIENT} --query "SYSTEM STOP MERGES" ${CLICKHOUSE_CLIENT} --query "SYSTEM DROP FILESYSTEM CACHE" ${CLICKHOUSE_CLIENT} --query "SYSTEM DROP MARK CACHE" ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM system.filesystem_cache" From 7eb5c9354bf0cdf9bad1e56e78bcf03fdae77444 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 8 May 2024 16:44:37 +0200 Subject: [PATCH 37/46] Revert "Revert "Fix index analysis for `DateTime64`"" --- src/Functions/FunctionsConversion.cpp | 2 +- .../02373_datetime64_monotonicity.queries | 57 ------------------- .../02373_datetime64_monotonicity.reference | 12 ++++ .../02373_datetime64_monotonicity.sh | 40 ++++++++++++- ...tetime64_constant_index_analysis.reference | 6 ++ ...147_datetime64_constant_index_analysis.sql | 12 ++++ 6 files changed, 70 insertions(+), 59 deletions(-) delete mode 100644 tests/queries/0_stateless/02373_datetime64_monotonicity.queries create mode 100644 tests/queries/0_stateless/03147_datetime64_constant_index_analysis.reference create mode 100644 tests/queries/0_stateless/03147_datetime64_constant_index_analysis.sql diff --git a/src/Functions/FunctionsConversion.cpp b/src/Functions/FunctionsConversion.cpp index a16ce136b9a7..0da84d6d224c 100644 --- a/src/Functions/FunctionsConversion.cpp +++ b/src/Functions/FunctionsConversion.cpp @@ -4853,7 +4853,7 @@ FunctionBasePtr createFunctionBaseCast( DataTypeUInt8, DataTypeUInt16, DataTypeUInt32, DataTypeUInt64, DataTypeUInt128, DataTypeUInt256, DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64, DataTypeInt128, DataTypeInt256, DataTypeFloat32, DataTypeFloat64, - DataTypeDate, DataTypeDate32, DataTypeDateTime, + DataTypeDate, DataTypeDate32, DataTypeDateTime, DataTypeDateTime64, DataTypeString>(return_type.get(), [&](auto & type) { monotonicity = FunctionTo>::Type::Monotonic::get; diff --git a/tests/queries/0_stateless/02373_datetime64_monotonicity.queries b/tests/queries/0_stateless/02373_datetime64_monotonicity.queries deleted file mode 100644 index 404e3391205b..000000000000 --- a/tests/queries/0_stateless/02373_datetime64_monotonicity.queries +++ /dev/null @@ -1,57 +0,0 @@ -drop table if exists dt64_monot_test; -drop table if exists dt64_monot_test_string; -CREATE TABLE dt64_monot_test(`date_time` DateTime64(3, 'Europe/Berlin'), `id` String) ENGINE = MergeTree PARTITION BY toDate(date_time, 'Europe/Berlin') ORDER BY date_time; -insert into dt64_monot_test select toDateTime64('2020-01-01 00:00:00.000',3)+number , '' from numbers(10); - -SELECT count() FROM dt64_monot_test WHERE toDateTime(date_time) >= toDateTime('2020-01-01 00:00:00') SETTINGS force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time, 3) >= '2020-01-01 00:00:01.111' SETTINGS force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time, 3) >= '2020-01-01 00:00:00.000' SETTINGS force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001', 3) SETTINGS force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001', 3, 'Europe/Berlin') SETTINGS force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001',6) SETTINGS force_index_by_date = 1; -- { serverError 277} - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') SETTINGS force_primary_key = 1; -- { serverError 277} - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) >= toDateTime64('2020-01-01 00:00:00.000001',6) SETTINGS force_primary_key = 1; -- { serverError 277} - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) <= toDateTime64('2020-01-01 00:00:00.000001',3, 'Europe/Berlin') settings force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) <= toDateTime64('2020-01-01 00:00:00.000001',3) settings force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) = toDateTime64('2020-01-01 00:00:00.000000',6); - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,3) = toDateTime64('2020-01-01 00:00:00.000000',6, 'Europe/Berlin'); - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) = toDateTime64('2020-01-01 00:00:00.000000',6) settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) = toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) > toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) >= toDateTime64('2020-01-01 00:00:00.000001',6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,6) >= toDateTime64('2020-01-01 00:00:00.000001',6) settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= toDateTime64('2020-01-01 00:00:00.000001',0, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= toDateTime64('2020-01-01 00:00:00.000001',0) settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= '2020-01-01 00:00:00' settings force_index_by_date = 1, force_primary_key = 1; - -SELECT count() FROM dt64_monot_test WHERE toDateTime64(date_time,0) >= '2020-01-01 00:00:01.1' settings force_index_by_date = 1, force_primary_key = 1; - -create table dt64_monot_test_string(date_time String, x String) Engine=MergeTree order by date_time; -insert into dt64_monot_test_string select '2020-01-01 00:00:00.000000001', '' from numbers(1); -insert into dt64_monot_test_string select '2020-01-01 00:00:00.000', '' from numbers(10); - -SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00.000000000'; -SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,3) = '2020-01-01 00:00:00.000000001'; -SELECT count() FROM dt64_monot_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00'; - -drop table dt64_monot_test; -drop table dt64_monot_test_string; diff --git a/tests/queries/0_stateless/02373_datetime64_monotonicity.reference b/tests/queries/0_stateless/02373_datetime64_monotonicity.reference index 935ee685cc99..dd7ddf43e704 100644 --- a/tests/queries/0_stateless/02373_datetime64_monotonicity.reference +++ b/tests/queries/0_stateless/02373_datetime64_monotonicity.reference @@ -4,6 +4,9 @@ Asia/Tehran 0 10 0 +9 +0 +9 10 1 1 @@ -27,6 +30,9 @@ UTC 10 10 10 +9 +10 +9 0 1 1 @@ -50,6 +56,9 @@ Canada/Atlantic 10 10 10 +9 +10 +9 0 1 1 @@ -73,6 +82,9 @@ Europe/Berlin 10 10 10 +9 +9 +9 1 1 1 diff --git a/tests/queries/0_stateless/02373_datetime64_monotonicity.sh b/tests/queries/0_stateless/02373_datetime64_monotonicity.sh index 0e0dc0ec22ab..7e852aa24b0b 100755 --- a/tests/queries/0_stateless/02373_datetime64_monotonicity.sh +++ b/tests/queries/0_stateless/02373_datetime64_monotonicity.sh @@ -7,6 +7,44 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) for tz in Asia/Tehran UTC Canada/Atlantic Europe/Berlin do echo "$tz" - TZ=$tz $CLICKHOUSE_LOCAL -mn < ${CUR_DIR}/02373_datetime64_monotonicity.queries + TZ=$tz $CLICKHOUSE_LOCAL --multiline --multiquery " + drop table if exists dt64_monotonicity_test; + drop table if exists dt64_monotonicity_test_string; + CREATE TABLE dt64_monotonicity_test (date_time DateTime64(3, 'Europe/Berlin'), id String) ENGINE = MergeTree PARTITION BY toDate(date_time, 'Europe/Berlin') ORDER BY date_time; + insert into dt64_monotonicity_test select toDateTime64('2020-01-01 00:00:00.000', 3) + number, '' from numbers(10); + + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime(date_time) >= toDateTime('2020-01-01 00:00:00') SETTINGS force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= '2020-01-01 00:00:01.111' SETTINGS force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= '2020-01-01 00:00:00.000' SETTINGS force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 3) SETTINGS force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 3, 'Europe/Berlin') SETTINGS force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 6) SETTINGS force_index_by_date = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 6, 'Europe/Berlin') SETTINGS force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) >= toDateTime64('2020-01-01 00:00:00.000001', 6) SETTINGS force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) <= toDateTime64('2020-01-01 00:00:00.000001', 3, 'Europe/Berlin') settings force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) <= toDateTime64('2020-01-01 00:00:00.000001', 3) settings force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) = toDateTime64('2020-01-01 00:00:00.000000', 6); + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 3) = toDateTime64('2020-01-01 00:00:00.000000', 6, 'Europe/Berlin'); + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) = toDateTime64('2020-01-01 00:00:00.000000', 6) settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) = toDateTime64('2020-01-01 00:00:00.000001', 6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) > toDateTime64('2020-01-01 00:00:00.000001', 6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) >= toDateTime64('2020-01-01 00:00:00.000001', 6, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 6) >= toDateTime64('2020-01-01 00:00:00.000001', 6) settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 0) >= toDateTime64('2020-01-01 00:00:00.000001', 0, 'Europe/Berlin') settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 0) >= toDateTime64('2020-01-01 00:00:00.000001', 0) settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 0) >= '2020-01-01 00:00:00' settings force_index_by_date = 1, force_primary_key = 1; + SELECT count() FROM dt64_monotonicity_test WHERE toDateTime64(date_time, 0) >= '2020-01-01 00:00:01.1' settings force_index_by_date = 1, force_primary_key = 1; + + create table dt64_monotonicity_test_string(date_time String, x String) Engine=MergeTree order by date_time; + insert into dt64_monotonicity_test_string select '2020-01-01 00:00:00.000000001', '' from numbers(1); + insert into dt64_monotonicity_test_string select '2020-01-01 00:00:00.000', '' from numbers(10); + + SELECT count() FROM dt64_monotonicity_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00.000000000'; + SELECT count() FROM dt64_monotonicity_test_string WHERE toDateTime64(date_time,3) = '2020-01-01 00:00:00.000000001'; + SELECT count() FROM dt64_monotonicity_test_string WHERE toDateTime64(date_time,9) = '2020-01-01 00:00:00'; + + drop table dt64_monotonicity_test; + drop table dt64_monotonicity_test_string; + " echo "" done diff --git a/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.reference b/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.reference new file mode 100644 index 000000000000..9abcce111362 --- /dev/null +++ b/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.reference @@ -0,0 +1,6 @@ +7385 +7385 +7385 +7385 +86401 +86401 diff --git a/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.sql b/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.sql new file mode 100644 index 000000000000..144478eb7215 --- /dev/null +++ b/tests/queries/0_stateless/03147_datetime64_constant_index_analysis.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS test; +CREATE TABLE test (d DateTime, PRIMARY KEY (d)); +INSERT INTO test SELECT toDateTime('2024-01-01') + number FROM numbers(1e6); +SET max_rows_to_read = 10000; +SELECT count() FROM test WHERE d <= '2024-01-01 02:03:04'; +SELECT count() FROM test WHERE d <= toDateTime('2024-01-01 02:03:04'); +SELECT count() FROM test WHERE d <= toDateTime64('2024-01-01 02:03:04', 0); +SELECT count() FROM test WHERE d <= toDateTime64('2024-01-01 02:03:04', 3); +SET max_rows_to_read = 100_000; +SELECT count() FROM test WHERE d <= '2024-01-02'; +SELECT count() FROM test WHERE d <= toDate('2024-01-02'); +DROP TABLE test; From c2b84431114cf71074923fd7e30f2891ec6a249c Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 7 May 2024 20:46:17 +0200 Subject: [PATCH 38/46] Fix for RabbitMQ --- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 154 ++++++++++-------- src/Storages/RabbitMQ/StorageRabbitMQ.h | 1 - .../integration/test_storage_rabbitmq/test.py | 2 +- 3 files changed, 91 insertions(+), 66 deletions(-) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 0358b2bbc662..e4b199921515 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -408,9 +408,7 @@ void StorageRabbitMQ::initRabbitMQ() /// Main exchange -> Bridge exchange -> ( Sharding exchange ) -> Queues -> Consumers - initExchange(*rabbit_channel); bindExchange(*rabbit_channel); - for (const auto i : collections::range(0, num_queues)) bindQueue(i + 1, *rabbit_channel); @@ -442,7 +440,7 @@ void StorageRabbitMQ::initRabbitMQ() } -void StorageRabbitMQ::initExchange(AMQP::TcpChannel & rabbit_channel) +void StorageRabbitMQ::bindExchange(AMQP::TcpChannel & rabbit_channel) { /// Exchange hierarchy: /// 1. Main exchange (defined with table settings - rabbitmq_exchange_name, rabbitmq_exchange_type). @@ -455,68 +453,78 @@ void StorageRabbitMQ::initExchange(AMQP::TcpChannel & rabbit_channel) /// 1. `durable` (survive RabbitMQ server restart) /// 2. `autodelete` (auto delete in case of queue bindings are dropped). + std::string error; + int error_code; rabbit_channel.declareExchange(exchange_name, exchange_type, AMQP::durable) .onError([&](const char * message) { + connection->getHandler().stopLoop(); /// This error can be a result of attempt to declare exchange if it was already declared but /// 1) with different exchange type. /// 2) with different exchange settings. - throw Exception(ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, - "Unable to declare exchange. Make sure specified exchange is not already declared. Error: {}", - std::string(message)); + error = "Unable to declare exchange. " + "Make sure specified exchange is not already declared. Error: " + std::string(message); + error_code = ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE; }); rabbit_channel.declareExchange(bridge_exchange, AMQP::fanout, AMQP::durable | AMQP::autodelete) .onError([&](const char * message) { + connection->getHandler().stopLoop(); /// This error is not supposed to happen as this exchange name is always unique to type and its settings. - throw Exception( - ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, - "Unable to declare bridge exchange ({}). Reason: {}", bridge_exchange, std::string(message)); + if (error.empty()) + { + error = fmt::format("Unable to declare bridge exchange ({}). Reason: {}", + bridge_exchange, std::string(message)); + error_code = ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE; + } }); - if (!hash_exchange) + if (hash_exchange) { - consumer_exchange = bridge_exchange; - return; - } + AMQP::Table binding_arguments; - AMQP::Table binding_arguments; + /// Default routing key property in case of hash exchange is a routing key, which is required to be an integer. + /// Support for arbitrary exchange type (i.e. arbitrary pattern of routing keys) requires to eliminate this dependency. + /// This settings changes hash property to message_id. + binding_arguments["hash-property"] = "message_id"; - /// Default routing key property in case of hash exchange is a routing key, which is required to be an integer. - /// Support for arbitrary exchange type (i.e. arbitrary pattern of routing keys) requires to eliminate this dependency. - /// This settings changes hash property to message_id. - binding_arguments["hash-property"] = "message_id"; + /// Declare hash exchange for sharding. + rabbit_channel.declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable | AMQP::autodelete, binding_arguments) + .onError([&](const char * message) + { + connection->getHandler().stopLoop(); + /// This error can be a result of same reasons as above for exchange_name, i.e. it will mean that sharding exchange name appeared + /// to be the same as some other exchange (which purpose is not for sharding). So probably actual error reason: queue_base parameter + /// is bad. + if (error.empty()) + { + error = fmt::format("Unable to declare sharding exchange ({}). Reason: {}", + sharding_exchange, std::string(message)); + error_code = ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE; + } + }); - /// Declare hash exchange for sharding. - rabbit_channel.declareExchange(sharding_exchange, AMQP::consistent_hash, AMQP::durable | AMQP::autodelete, binding_arguments) - .onError([&](const char * message) - { - /// This error can be a result of same reasons as above for exchange_name, i.e. it will mean that sharding exchange name appeared - /// to be the same as some other exchange (which purpose is not for sharding). So probably actual error reason: queue_base parameter - /// is bad. - throw Exception( - ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE, - "Unable to declare sharding exchange ({}). Reason: {}", sharding_exchange, std::string(message)); - }); + rabbit_channel.bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) + .onError([&](const char * message) + { + connection->getHandler().stopLoop(); + if (error.empty()) + { + error = fmt::format( + "Unable to bind bridge exchange ({}) to sharding exchange ({}). Reason: {}", + bridge_exchange, sharding_exchange, std::string(message)); + error_code = ErrorCodes::CANNOT_DECLARE_RABBITMQ_EXCHANGE; + } + }); - rabbit_channel.bindExchange(bridge_exchange, sharding_exchange, routing_keys[0]) - .onError([&](const char * message) + consumer_exchange = sharding_exchange; + } + else { - throw Exception( - ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, - "Unable to bind bridge exchange ({}) to sharding exchange ({}). Reason: {}", - bridge_exchange, - sharding_exchange, - std::string(message)); - }); - - consumer_exchange = sharding_exchange; -} - + consumer_exchange = bridge_exchange; + } -void StorageRabbitMQ::bindExchange(AMQP::TcpChannel & rabbit_channel) -{ size_t bound_keys = 0; if (exchange_type == AMQP::ExchangeType::headers) @@ -533,10 +541,10 @@ void StorageRabbitMQ::bindExchange(AMQP::TcpChannel & rabbit_channel) .onSuccess([&]() { connection->getHandler().stopLoop(); }) .onError([&](const char * message) { - throw Exception( - ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, - "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", - exchange_name, bridge_exchange, std::string(message)); + connection->getHandler().stopLoop(); + error = fmt::format("Unable to bind exchange {} to bridge exchange ({}). Reason: {}", + exchange_name, bridge_exchange, std::string(message)); + error_code = ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE; }); } else if (exchange_type == AMQP::ExchangeType::fanout || exchange_type == AMQP::ExchangeType::consistent_hash) @@ -545,10 +553,13 @@ void StorageRabbitMQ::bindExchange(AMQP::TcpChannel & rabbit_channel) .onSuccess([&]() { connection->getHandler().stopLoop(); }) .onError([&](const char * message) { - throw Exception( - ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, - "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", - exchange_name, bridge_exchange, std::string(message)); + connection->getHandler().stopLoop(); + if (error.empty()) + { + error = fmt::format("Unable to bind exchange {} to bridge exchange ({}). Reason: {}", + exchange_name, bridge_exchange, std::string(message)); + error_code = ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE; + } }); } else @@ -564,20 +575,26 @@ void StorageRabbitMQ::bindExchange(AMQP::TcpChannel & rabbit_channel) }) .onError([&](const char * message) { - throw Exception( - ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE, - "Unable to bind exchange {} to bridge exchange ({}). Reason: {}", - exchange_name, bridge_exchange, std::string(message)); + connection->getHandler().stopLoop(); + if (error.empty()) + { + error = fmt::format("Unable to bind exchange {} to bridge exchange ({}). Reason: {}", + exchange_name, bridge_exchange, std::string(message)); + error_code = ErrorCodes::CANNOT_BIND_RABBITMQ_EXCHANGE; + } }); } } connection->getHandler().startBlockingLoop(); + if (!error.empty()) + throw Exception(error_code, "{}", error); } void StorageRabbitMQ::bindQueue(size_t queue_id, AMQP::TcpChannel & rabbit_channel) { + std::string error; auto success_callback = [&](const std::string & queue_name, int msgcount, int /* consumercount */) { queues.emplace_back(queue_name); @@ -594,23 +611,26 @@ void StorageRabbitMQ::bindQueue(size_t queue_id, AMQP::TcpChannel & rabbit_chann .onSuccess([&] { connection->getHandler().stopLoop(); }) .onError([&](const char * message) { - throw Exception( - ErrorCodes::CANNOT_CREATE_RABBITMQ_QUEUE_BINDING, - "Failed to create queue binding for exchange {}. Reason: {}", exchange_name, std::string(message)); + connection->getHandler().stopLoop(); + error = fmt::format("Failed to create queue binding for exchange {}. Reason: {}", + exchange_name, std::string(message)); }); }; auto error_callback([&](const char * message) { + connection->getHandler().stopLoop(); /* This error is most likely a result of an attempt to declare queue with different settings if it was declared before. So for a * given queue name either deadletter_exchange parameter changed or queue_size changed, i.e. table was declared with different * max_block_size parameter. Solution: client should specify a different queue_base parameter or manually delete previously * declared queues via any of the various cli tools. */ - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Failed to declare queue. Probably queue settings are conflicting: " - "max_block_size, deadletter_exchange. Attempt specifying differently those settings " - "or use a different queue_base or manually delete previously declared queues, " - "which were declared with the same names. ERROR reason: {}", std::string(message)); + if (error.empty()) + error = fmt::format( + "Failed to declare queue. Probably queue settings are conflicting: " + "max_block_size, deadletter_exchange. Attempt specifying differently those settings " + "or use a different queue_base or manually delete previously declared queues, " + "which were declared with the same names. ERROR reason: {}", std::string(message)); }); AMQP::Table queue_settings; @@ -648,6 +668,8 @@ void StorageRabbitMQ::bindQueue(size_t queue_id, AMQP::TcpChannel & rabbit_chann /// and deleting queues should not take place. rabbit_channel.declareQueue(queue_name, AMQP::durable, queue_settings).onSuccess(success_callback).onError(error_callback); connection->getHandler().startBlockingLoop(); + if (!error.empty()) + throw Exception(ErrorCodes::CANNOT_CREATE_RABBITMQ_QUEUE_BINDING, "{}", error); } @@ -671,6 +693,7 @@ void StorageRabbitMQ::unbindExchange() stopLoop(); looping_task->deactivate(); + std::string error; auto rabbit_channel = connection->createChannel(); rabbit_channel->removeExchange(bridge_exchange) @@ -680,11 +703,14 @@ void StorageRabbitMQ::unbindExchange() }) .onError([&](const char * message) { - throw Exception(ErrorCodes::CANNOT_REMOVE_RABBITMQ_EXCHANGE, "Unable to remove exchange. Reason: {}", std::string(message)); + connection->getHandler().stopLoop(); + error = fmt::format("Unable to remove exchange. Reason: {}", std::string(message)); }); connection->getHandler().startBlockingLoop(); rabbit_channel->close(); + if (!error.empty()) + throw Exception(ErrorCodes::CANNOT_REMOVE_RABBITMQ_EXCHANGE, "{}", error); } catch (...) { diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index 59fab7cd1ea2..b8fab5825e41 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -183,7 +183,6 @@ class StorageRabbitMQ final: public IStorage, WithContext void initRabbitMQ(); void cleanupRabbitMQ() const; - void initExchange(AMQP::TcpChannel & rabbit_channel); void bindExchange(AMQP::TcpChannel & rabbit_channel); void bindQueue(size_t queue_id, AMQP::TcpChannel & rabbit_channel); diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index f8e785d5ce68..23a95d5dd71c 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2606,7 +2606,7 @@ def test_rabbitmq_bad_args(rabbitmq_cluster): connection = pika.BlockingConnection(parameters) channel = connection.channel() channel.exchange_declare(exchange="f", exchange_type="fanout") - instance.query_and_get_error( + assert "Unable to declare exchange" in instance.query_and_get_error( """ CREATE TABLE test.drop (key UInt64, value UInt64) ENGINE = RabbitMQ From 071ff50d204bf5c7454c760fac3966dbc77c6145 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 8 May 2024 17:08:12 +0200 Subject: [PATCH 39/46] Fix data race inside distributed sink --- src/Storages/Distributed/DistributedSink.cpp | 9 +++++++++ src/Storages/Distributed/DistributedSink.h | 2 ++ 2 files changed, 11 insertions(+) diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index b89a8d7bcfd5..96313e333bf8 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -436,6 +436,10 @@ DistributedSink::runWritingJob(JobReplica & job, const Block & current_block, si void DistributedSink::writeSync(const Block & block) { + std::lock_guard lock(execution_mutex); + if (isCancelled()) + return; + OpenTelemetry::SpanHolder span(__PRETTY_FUNCTION__); const Settings & settings = context->getSettingsRef(); @@ -537,6 +541,10 @@ void DistributedSink::onFinish() LOG_DEBUG(log, "It took {} sec. to insert {} blocks, {} rows per second. {}", elapsed, inserted_blocks, inserted_rows / elapsed, getCurrentStateDescription()); }; + std::lock_guard lock(execution_mutex); + if (isCancelled()) + return; + /// Pool finished means that some exception had been thrown before, /// and scheduling new jobs will return "Cannot schedule a task" error. if (insert_sync && pool && !pool->finished()) @@ -587,6 +595,7 @@ void DistributedSink::onFinish() void DistributedSink::onCancel() { + std::lock_guard lock(execution_mutex); if (pool && !pool->finished()) { try diff --git a/src/Storages/Distributed/DistributedSink.h b/src/Storages/Distributed/DistributedSink.h index 7a9e89c9e94b..a4c95633595b 100644 --- a/src/Storages/Distributed/DistributedSink.h +++ b/src/Storages/Distributed/DistributedSink.h @@ -113,6 +113,8 @@ class DistributedSink : public SinkToStorage std::optional pool; ThrottlerPtr throttler; + std::mutex execution_mutex; + struct JobReplica { JobReplica() = default; From bbb8cc0e9321ed8091c54ad1302fab0fe786930b Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 8 May 2024 18:01:50 +0200 Subject: [PATCH 40/46] Fix azure tests run on master --- tests/ci/ci.py | 9 ++++++--- tests/ci/test_ci_options.py | 7 ++++--- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index a95ada628ff1..6d821af32c12 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -849,6 +849,7 @@ def apply( jobs_to_do: List[str], jobs_to_skip: List[str], jobs_params: Dict[str, Dict[str, Any]], + pr_info: PRInfo, ) -> Tuple[List[str], List[str], Dict[str, Dict[str, Any]]]: """ Applies specified options on CI Run Config @@ -948,7 +949,8 @@ def apply( jobs_params[job] = { "batches": list(range(num_batches)), "num_batches": num_batches, - "run_if_ci_option_include_set": job_config.run_by_ci_option, + "run_if_ci_option_include_set": job_config.run_by_ci_option + and pr_info.is_pr, } # 4. Handle "batch_" tags @@ -1439,7 +1441,8 @@ def _configure_jobs( jobs_params[job] = { "batches": batches_to_do, "num_batches": num_batches, - "run_if_ci_option_include_set": job_config.run_by_ci_option, + "run_if_ci_option_include_set": job_config.run_by_ci_option + and pr_info.is_pr, } elif add_to_skip: # treat job as being skipped only if it's controlled by digest @@ -1464,7 +1467,7 @@ def _configure_jobs( ] jobs_to_do, jobs_to_skip, jobs_params = ci_options.apply( - jobs_to_do, jobs_to_skip, jobs_params + jobs_to_do, jobs_to_skip, jobs_params, pr_info ) return { diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index 3d9c02822bdc..0f10f7d4f85f 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -4,6 +4,7 @@ import unittest from ci import CiOptions +from pr_info import PRInfo _TEST_BODY_1 = """ #### Run only: @@ -164,7 +165,7 @@ def test_options_applied(self): } } jobs_to_do, jobs_to_skip, job_params = ci_options.apply( - jobs_to_do, jobs_to_skip, job_params + jobs_to_do, jobs_to_skip, job_params, PRInfo() ) self.assertCountEqual( jobs_to_do, @@ -196,7 +197,7 @@ def test_options_applied_2(self): jobs_to_skip = [] job_params = {} jobs_to_do, jobs_to_skip, job_params = ci_options.apply( - jobs_to_do, jobs_to_skip, job_params + jobs_to_do, jobs_to_skip, job_params, PRInfo() ) self.assertCountEqual( jobs_to_do, @@ -231,7 +232,7 @@ def test_options_applied_3(self): job_params[job] = {"run_if_ci_option_include_set": False} jobs_to_do, jobs_to_skip, job_params = ci_options.apply( - jobs_to_do, jobs_to_skip, job_params + jobs_to_do, jobs_to_skip, job_params, PRInfo() ) self.assertNotIn( "Stateless tests (azure, asan)", From 8d2fcbd91de9311286b1bc8fff78dc415d2b6f44 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 May 2024 18:50:46 +0200 Subject: [PATCH 41/46] Remove check --- tests/config/install.sh | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/config/install.sh b/tests/config/install.sh index 33dcac9d2c73..6536683b6c2e 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -181,11 +181,8 @@ elif [[ "$USE_AZURE_STORAGE_FOR_MERGE_TREE" == "1" ]]; then ln -sf $SRC_PATH/config.d/azure_storage_policy_by_default.xml $DEST_SERVER_PATH/config.d/ fi -ARM="aarch64" -OS="$(uname -m)" if [[ -n "$EXPORT_S3_STORAGE_POLICIES" ]]; then - echo "$OS" - if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] || [[ "$OS" == "$ARM" ]]; then + if [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then echo "Azure configuration will not be added" else echo "Adding azure configuration" From 47473d77abca5887914d4f703961ffbbb72000cf Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 8 May 2024 20:31:02 +0200 Subject: [PATCH 42/46] Find a proper commit for cumulative `A Sync` status --- tests/ci/ci.py | 64 +++++++++++++++++++++++++++++++++++++------------- 1 file changed, 48 insertions(+), 16 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 01179f847638..fe68dc32b793 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -14,6 +14,8 @@ from pathlib import Path from typing import Any, Dict, List, Optional, Sequence, Set, Tuple, Union +from github.CommitStatus import CommitStatus + import docker_images_helper import upload_result_helper from build_check import get_release_or_pr @@ -1908,6 +1910,51 @@ def _get_ext_check_name(check_name: str) -> str: return check_name_with_group +def update_upstream_a_sync( + pr_info: PRInfo, + gh: GitHub, + mergeable_status: CommitStatus, +) -> None: + pr_number = int(pr_info.head_ref.split("/pr/", maxsplit=1)[1]) + upstream_repo = gh.get_repo(GITHUB_UPSTREAM_REPOSITORY) + upstream_pr = upstream_repo.get_pull(pr_number) + sync_repo = gh.get_repo(GITHUB_REPOSITORY) + sync_pr = sync_repo.get_pull(pr_info.number) + # Find the commit that is in both repos, upstream and cloud + sync_commits = sync_pr.get_commits().reversed + upstream_commits = upstream_pr.get_commits() + # Github objects are compared by _url attribute. We can't compare them directly and + # should compare commits by SHA1 + upstream_shas = [uc.sha for uc in upstream_commits] + found = False + for commit in sync_commits: + try: + idx = upstream_shas.index(commit.sha) + found = True + upstream_commit = upstream_commits[idx] + except ValueError: + continue + + if not found: + print( + "No same commits found in upstream and sync repo, most probably force-push" + ) + return + + post_commit_status( + upstream_commit, + get_status(mergeable_status.state), + "", # let's won't expose any urls from cloud + mergeable_status.description, + StatusNames.SYNC, + ) + trigger_mergeable_check( + upstream_commit, + get_commit_filtered_statuses(upstream_commit), + True, + ) + + def main() -> int: logging.basicConfig(level=logging.INFO) exit_code = 0 @@ -2204,23 +2251,8 @@ def main() -> int: and mergeable_status and GITHUB_REPOSITORY != GITHUB_UPSTREAM_REPOSITORY ): + update_upstream_a_sync(pr_info, gh, mergeable_status) pr_number = int(pr_info.head_ref.split("/pr/", maxsplit=1)[1]) - upstream_repo = gh.get_repo(GITHUB_UPSTREAM_REPOSITORY) - head_sha = upstream_repo.get_pull(pr_number).head.sha - upstream_commit = upstream_repo.get_commit(head_sha) - post_commit_status( - upstream_commit, - get_status(mergeable_status.state), - "", # let's won't expose any urls from cloud - mergeable_status.description, - StatusNames.SYNC, - ) - trigger_mergeable_check( - upstream_commit, - get_commit_filtered_statuses(upstream_commit), - True, - ) - prepared_events = prepare_tests_results_for_clickhouse( pr_info, [], From afda3c24e48a1ff1375c3a84e958273fe92d015b Mon Sep 17 00:00:00 2001 From: Julia Kartseva Date: Wed, 8 May 2024 19:41:41 +0000 Subject: [PATCH 43/46] Add no-s3-storage tag to local_plain_rewritable ut This blacklists the test in the distributed cache build. --- tests/queries/0_stateless/03008_local_plain_rewritable.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03008_local_plain_rewritable.sh b/tests/queries/0_stateless/03008_local_plain_rewritable.sh index 07fd013c911b..77bc6763fd82 100755 --- a/tests/queries/0_stateless/03008_local_plain_rewritable.sh +++ b/tests/queries/0_stateless/03008_local_plain_rewritable.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-random-settings, no-replicated-database, no-shared-merge-tree +# Tags: no-random-settings, no-s3-storage, no-replicated-database, no-shared-merge-tree # Tag no-random-settings: enable after root causing flakiness CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) From 27551ca3c7ec98eed6e3d65a6b049838bc01ff83 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 9 May 2024 04:08:21 +0300 Subject: [PATCH 44/46] Add `jwcrypto` to integration tests runner --- docker/test/integration/runner/Dockerfile | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 8297a7100d15..23d8a37d8226 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -101,7 +101,8 @@ RUN python3 -m pip install --no-cache-dir \ retry==0.9.2 \ bs4==0.0.2 \ lxml==5.1.0 \ - urllib3==2.0.7 + urllib3==2.0.7 \ + jwcrypto==1.5.6 # bs4, lxml are for cloud tests, do not delete # Hudi supports only spark 3.3.*, not 3.4 From f294f6d16fd6d4aca051ae2ac01d71925eed9537 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 9 May 2024 12:48:44 +0200 Subject: [PATCH 45/46] Move update_upstream_sync_status to commit_status_helper --- tests/ci/ci.py | 62 +++++--------------------------- tests/ci/commit_status_helper.py | 57 ++++++++++++++++++++++++++++- 2 files changed, 65 insertions(+), 54 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index fe68dc32b793..6f72d6b43c21 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -14,8 +14,6 @@ from pathlib import Path from typing import Any, Dict, List, Optional, Sequence, Set, Tuple, Union -from github.CommitStatus import CommitStatus - import docker_images_helper import upload_result_helper from build_check import get_release_or_pr @@ -34,11 +32,10 @@ RerunHelper, format_description, get_commit, - get_commit_filtered_statuses, post_commit_status, set_status_comment, - trigger_mergeable_check, update_mergeable_check, + update_upstream_sync_status, ) from digest_helper import DockerDigester, JobDigester from env_helper import ( @@ -57,7 +54,7 @@ from git_helper import Runner as GitRunner from github_helper import GitHub from pr_info import PRInfo -from report import ERROR, SUCCESS, BuildResult, JobReport, get_status +from report import ERROR, SUCCESS, BuildResult, JobReport from s3_helper import S3Helper from synchronizer_utils import SYNC_BRANCH_PREFIX from version_helper import get_version_from_repo @@ -1910,51 +1907,6 @@ def _get_ext_check_name(check_name: str) -> str: return check_name_with_group -def update_upstream_a_sync( - pr_info: PRInfo, - gh: GitHub, - mergeable_status: CommitStatus, -) -> None: - pr_number = int(pr_info.head_ref.split("/pr/", maxsplit=1)[1]) - upstream_repo = gh.get_repo(GITHUB_UPSTREAM_REPOSITORY) - upstream_pr = upstream_repo.get_pull(pr_number) - sync_repo = gh.get_repo(GITHUB_REPOSITORY) - sync_pr = sync_repo.get_pull(pr_info.number) - # Find the commit that is in both repos, upstream and cloud - sync_commits = sync_pr.get_commits().reversed - upstream_commits = upstream_pr.get_commits() - # Github objects are compared by _url attribute. We can't compare them directly and - # should compare commits by SHA1 - upstream_shas = [uc.sha for uc in upstream_commits] - found = False - for commit in sync_commits: - try: - idx = upstream_shas.index(commit.sha) - found = True - upstream_commit = upstream_commits[idx] - except ValueError: - continue - - if not found: - print( - "No same commits found in upstream and sync repo, most probably force-push" - ) - return - - post_commit_status( - upstream_commit, - get_status(mergeable_status.state), - "", # let's won't expose any urls from cloud - mergeable_status.description, - StatusNames.SYNC, - ) - trigger_mergeable_check( - upstream_commit, - get_commit_filtered_statuses(upstream_commit), - True, - ) - - def main() -> int: logging.basicConfig(level=logging.INFO) exit_code = 0 @@ -2251,15 +2203,19 @@ def main() -> int: and mergeable_status and GITHUB_REPOSITORY != GITHUB_UPSTREAM_REPOSITORY ): - update_upstream_a_sync(pr_info, gh, mergeable_status) - pr_number = int(pr_info.head_ref.split("/pr/", maxsplit=1)[1]) + upstream_pr_number = int( + pr_info.head_ref.split("/pr/", maxsplit=1)[1] + ) + update_upstream_sync_status( + upstream_pr_number, pr_info.number, gh, mergeable_status + ) prepared_events = prepare_tests_results_for_clickhouse( pr_info, [], job_report.status, 0, job_report.start_time, - f"https://github.com/ClickHouse/ClickHouse/pull/{pr_number}", + f"https://github.com/ClickHouse/ClickHouse/pull/{upstream_pr_number}", StatusNames.SYNC, ) prepared_events[0]["test_context_raw"] = args.job_name diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 6421ac6f0dbc..6f633cb44100 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -18,7 +18,12 @@ from github.Repository import Repository from ci_config import CHECK_DESCRIPTIONS, REQUIRED_CHECKS, CheckDescription, StatusNames -from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL, TEMP_PATH +from env_helper import ( + GITHUB_REPOSITORY, + GITHUB_RUN_URL, + GITHUB_UPSTREAM_REPOSITORY, + TEMP_PATH, +) from lambda_shared_package.lambda_shared.pr import Labels from pr_info import PRInfo from report import ( @@ -29,6 +34,7 @@ StatusType, TestResult, TestResults, + get_status, get_worst_status, ) from s3_helper import S3Helper @@ -500,3 +506,52 @@ def trigger_mergeable_check( return set_mergeable_check(commit, description, state, hide_url) return mergeable_status + + +def update_upstream_sync_status( + upstream_pr_number: int, + sync_pr_number: int, + gh: Github, + mergeable_status: CommitStatus, +) -> None: + upstream_repo = gh.get_repo(GITHUB_UPSTREAM_REPOSITORY) + upstream_pr = upstream_repo.get_pull(upstream_pr_number) + sync_repo = gh.get_repo(GITHUB_REPOSITORY) + sync_pr = sync_repo.get_pull(sync_pr_number) + # Find the commit that is in both repos, upstream and cloud + sync_commits = sync_pr.get_commits().reversed + upstream_commits = upstream_pr.get_commits() + # Github objects are compared by _url attribute. We can't compare them directly and + # should compare commits by SHA1 + upstream_shas = [uc.sha for uc in upstream_commits] + found = False + for commit in sync_commits: + try: + idx = upstream_shas.index(commit.sha) + found = True + upstream_commit = upstream_commits[idx] + break + except ValueError: + continue + + if not found: + logging.info( + "No same commits found in upstream and sync repo, most probably force-push" + ) + logging.info("Commits in upstream PR:\n %s", ", ".join(upstream_shas)) + sync_shas = [uc.sha for uc in upstream_commits] + logging.info("Commits in sync PR:\n %s", ", ".join(reversed(sync_shas))) + return + + post_commit_status( + upstream_commit, + get_status(mergeable_status.state), + "", # let's won't expose any urls from cloud + mergeable_status.description, + StatusNames.SYNC, + ) + trigger_mergeable_check( + upstream_commit, + get_commit_filtered_statuses(upstream_commit), + True, + ) From 38604eb3bc20c7f23e9975fa373ebb4fd05a4972 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 9 May 2024 12:57:30 +0200 Subject: [PATCH 46/46] Add more logging about the status will be post --- tests/ci/commit_status_helper.py | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 6f633cb44100..0b51d98b4793 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -524,6 +524,9 @@ def update_upstream_sync_status( # Github objects are compared by _url attribute. We can't compare them directly and # should compare commits by SHA1 upstream_shas = [uc.sha for uc in upstream_commits] + logging.info("Commits in upstream PR:\n %s", ", ".join(upstream_shas)) + sync_shas = [uc.sha for uc in upstream_commits] + logging.info("Commits in sync PR:\n %s", ", ".join(reversed(sync_shas))) found = False for commit in sync_commits: try: @@ -536,16 +539,21 @@ def update_upstream_sync_status( if not found: logging.info( - "No same commits found in upstream and sync repo, most probably force-push" + "There's no same commits in upstream and sync PRs, probably force-push" ) - logging.info("Commits in upstream PR:\n %s", ", ".join(upstream_shas)) - sync_shas = [uc.sha for uc in upstream_commits] - logging.info("Commits in sync PR:\n %s", ", ".join(reversed(sync_shas))) return + sync_status = get_status(mergeable_status.state) + logging.info( + "Using commit %s to post the %s status `%s`: [%s]", + upstream_commit.sha, + sync_status, + StatusNames.SYNC, + mergeable_status.description, + ) post_commit_status( upstream_commit, - get_status(mergeable_status.state), + sync_status, "", # let's won't expose any urls from cloud mergeable_status.description, StatusNames.SYNC,