From 2d1f45bb985debfb20037cb5f7dba6fb4f6903d8 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 11 Jul 2024 08:32:00 +0000 Subject: [PATCH 01/32] Remove redundant code --- src/Client/IConnections.h | 2 -- src/Client/MultiplexedConnections.cpp | 11 +++++------ src/Interpreters/ClientInfo.cpp | 4 ++-- src/Interpreters/ClientInfo.h | 2 +- src/Interpreters/Context.cpp | 7 ------- src/Interpreters/Context.h | 1 - src/Processors/QueryPlan/ReadFromRemote.cpp | 13 ++++++------- 7 files changed, 14 insertions(+), 26 deletions(-) diff --git a/src/Client/IConnections.h b/src/Client/IConnections.h index ebc71511834..09211de53b0 100644 --- a/src/Client/IConnections.h +++ b/src/Client/IConnections.h @@ -54,8 +54,6 @@ public: struct ReplicaInfo { - bool collaborate_with_initiator{false}; - size_t all_replicas_count{0}; size_t number_of_current_replica{0}; }; diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index bcef286ecbc..866338bee8d 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -141,13 +141,12 @@ void MultiplexedConnections::sendQuery( modified_settings.group_by_two_level_threshold = 0; modified_settings.group_by_two_level_threshold_bytes = 0; } + } - if (replica_info) - { - client_info.collaborate_with_initiator = true; - client_info.count_participating_replicas = replica_info->all_replicas_count; - client_info.number_of_current_replica = replica_info->number_of_current_replica; - } + if (replica_info) + { + client_info.collaborate_with_initiator = true; + client_info.number_of_current_replica = replica_info->number_of_current_replica; } /// FIXME: Remove once we will make `allow_experimental_analyzer` obsolete setting. diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index ce1efb61cc0..39fdef23baa 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -95,7 +95,7 @@ void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const if (server_protocol_revision >= DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS) { writeVarUInt(static_cast(collaborate_with_initiator), out); - writeVarUInt(count_participating_replicas, out); + writeVarUInt(obsolete_count_participating_replicas, out); writeVarUInt(number_of_current_replica, out); } } @@ -185,7 +185,7 @@ void ClientInfo::read(ReadBuffer & in, UInt64 client_protocol_revision) UInt64 value; readVarUInt(value, in); collaborate_with_initiator = static_cast(value); - readVarUInt(count_participating_replicas, in); + readVarUInt(obsolete_count_participating_replicas, in); readVarUInt(number_of_current_replica, in); } } diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 3054667e264..ca32b4c5cfa 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -127,7 +127,7 @@ public: /// For parallel processing on replicas bool collaborate_with_initiator{false}; - UInt64 count_participating_replicas{0}; + UInt64 obsolete_count_participating_replicas{0}; UInt64 number_of_current_replica{0}; enum class BackgroundOperationType : uint8_t diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index f2626696492..0d60f7dc5f4 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4941,13 +4941,6 @@ void Context::setConnectionClientVersion(UInt64 client_version_major, UInt64 cli client_info.connection_tcp_protocol_version = client_tcp_protocol_version; } -void Context::setReplicaInfo(bool collaborate_with_initiator, size_t all_replicas_count, size_t number_of_current_replica) -{ - client_info.collaborate_with_initiator = collaborate_with_initiator; - client_info.count_participating_replicas = all_replicas_count; - client_info.number_of_current_replica = number_of_current_replica; -} - void Context::increaseDistributedDepth() { ++client_info.distributed_depth; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 8c5492bcbc8..5cd1153ab44 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -696,7 +696,6 @@ public: void setInitialQueryStartTime(std::chrono::time_point initial_query_start_time); void setQuotaClientKey(const String & quota_key); void setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); - void setReplicaInfo(bool collaborate_with_initiator, size_t all_replicas_count, size_t number_of_current_replica); void increaseDistributedDepth(); const OpenTelemetry::TracingContext & getClientTraceContext() const { return client_info.client_trace_context; } OpenTelemetry::TracingContext & getClientTraceContext() { return client_info.client_trace_context; } diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 84c2515e8ca..8ebd95f27e5 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -411,8 +411,8 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); const auto & shard = cluster->getShardsInfo().at(0); - size_t all_replicas_count = current_settings.max_parallel_replicas; - if (all_replicas_count > shard.getAllNodeCount()) + size_t max_replicas_to_use = current_settings.max_parallel_replicas; + if (max_replicas_to_use > shard.getAllNodeCount()) { LOG_INFO( getLogger("ReadFromParallelRemoteReplicasStep"), @@ -420,14 +420,14 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder "Will use the latter number to execute the query.", current_settings.max_parallel_replicas, shard.getAllNodeCount()); - all_replicas_count = shard.getAllNodeCount(); + max_replicas_to_use = shard.getAllNodeCount(); } std::vector shuffled_pool; - if (all_replicas_count < shard.getAllNodeCount()) + if (max_replicas_to_use < shard.getAllNodeCount()) { shuffled_pool = shard.pool->getShuffledPools(current_settings); - shuffled_pool.resize(all_replicas_count); + shuffled_pool.resize(max_replicas_to_use); } else { @@ -437,11 +437,10 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func); } - for (size_t i=0; i < all_replicas_count; ++i) + for (size_t i=0; i < max_replicas_to_use; ++i) { IConnections::ReplicaInfo replica_info { - .all_replicas_count = all_replicas_count, /// we should use this number specifically because efficiency of data distribution by consistent hash depends on it. .number_of_current_replica = i, }; From f85be2b452e9d62f766cee3d170dacd1906e04b7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 11 Jul 2024 12:17:59 +0000 Subject: [PATCH 02/32] Add --memory-usage option to client in non interactive mode --- src/Client/ClientBase.cpp | 22 ++++++++++++++++++++-- src/Common/ProgressIndication.h | 10 +++++----- 2 files changed, 25 insertions(+), 7 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5d472ba99b9..b5de348d583 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2069,9 +2069,18 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin progress_indication.writeFinalProgress(); output_stream << std::endl << std::endl; } - else if (getClientConfiguration().getBool("print-time-to-stderr", false)) + else { - error_stream << progress_indication.elapsedSeconds() << "\n"; + const auto & config = getClientConfiguration(); + if (config.getBool("print-time-to-stderr", false)) + error_stream << progress_indication.elapsedSeconds() << "\n"; + + const auto & print_memory_mode = config.getString("print-memory-to-stderr", ""); + auto peak_memeory_usage = std::max(progress_indication.getMemoryUsage().peak, 0); + if (print_memory_mode == "default") + error_stream << peak_memeory_usage << "\n"; + else if (print_memory_mode == "readable") + error_stream << formatReadableSizeWithBinarySuffix(peak_memeory_usage) << "\n"; } if (!is_interactive && getClientConfiguration().getBool("print-num-processed-rows", false)) @@ -3035,6 +3044,7 @@ void ClientBase::init(int argc, char ** argv) ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") ("wait_for_suggestions_to_load", "Load suggestion data synchonously.") ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") + ("memory-usage", po::value()->implicit_value("default")->default_value("none"), "print memory usage to stderr in non-interactive mode (for benchmarks). Values: 'none', 'default', 'readable'") ("echo", "in batch mode, print query before execution") @@ -3120,6 +3130,14 @@ void ClientBase::init(int argc, char ** argv) /// Output execution time to stderr in batch mode. if (options.count("time")) getClientConfiguration().setBool("print-time-to-stderr", true); + if (options.count("memory-usage")) + { + const auto & memory_usage_mode = options["memory-usage"].as(); + if (memory_usage_mode != "none" && memory_usage_mode != "default" && memory_usage_mode != "readable") + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown memory-usage mode: {}", memory_usage_mode); + getClientConfiguration().setString("print-memory-to-stderr", memory_usage_mode); + } + if (options.count("query")) queries = options["query"].as>(); if (options.count("query_id")) diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index ae39fb49bcc..c7f61a7fba9 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -71,11 +71,6 @@ public: /// How much seconds passed since query execution start. double elapsedSeconds() const { return getElapsedNanoseconds() / 1e9; } - void updateThreadEventData(HostToTimesMap & new_hosts_data); - -private: - double getCPUUsage(); - struct MemoryUsage { UInt64 total = 0; @@ -85,6 +80,11 @@ private: MemoryUsage getMemoryUsage() const; + void updateThreadEventData(HostToTimesMap & new_hosts_data); + +private: + double getCPUUsage(); + UInt64 getElapsedNanoseconds() const; /// This flag controls whether to show the progress bar. We start showing it after From b9cd1bddd7c8612e3b43172ea8670a7e904fe237 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 11 Jul 2024 12:20:37 +0000 Subject: [PATCH 03/32] upd doc --- docs/en/interfaces/cli.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index e18ff6f1a3f..63b7353d092 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -185,6 +185,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--format, -f` – Use the specified default format to output the result. - `--vertical, -E` – If specified, use the [Vertical format](../interfaces/formats.md#vertical) by default to output the result. This is the same as `–format=Vertical`. In this format, each value is printed on a separate line, which is helpful when displaying wide tables. - `--time, -t` – If specified, print the query execution time to ‘stderr’ in non-interactive mode. +- `--memory-usage` – If specified, print the query memory usage to ‘stderr’ in non-interactive mode. - `--stacktrace` – If specified, also print the stack trace if an exception occurs. - `--config-file` – The name of the configuration file. - `--secure` – If specified, will connect to server over secure connection (TLS). You might need to configure your CA certificates in the [configuration file](#configuration_files). The available configuration settings are the same as for [server-side TLS configuration](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl). @@ -339,7 +340,7 @@ clickhouse-client clickhouse://some_user%40some_mail.com@localhost:9000 Connect to one of provides hosts: `192.168.1.15`, `192.168.1.25`. ``` bash -clickhouse-client clickhouse://192.168.1.15,192.168.1.25 +clickhouse-client clickhouse://192.168.1.15,192.168.1.25 ``` ### Configuration Files {#configuration_files} @@ -367,7 +368,7 @@ Example of a config file: ``` Or the same config in a YAML format: - + ```yaml user: username password: 'password' From 05c3692e1dfcacffc9de62bac9a53a0e28f4bea7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 11 Jul 2024 12:22:03 +0000 Subject: [PATCH 04/32] add 03203_client_benchmark_options --- .../03203_client_benchmark_options.reference | 6 ++++++ .../0_stateless/03203_client_benchmark_options.sh | 13 +++++++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/03203_client_benchmark_options.reference create mode 100755 tests/queries/0_stateless/03203_client_benchmark_options.sh diff --git a/tests/queries/0_stateless/03203_client_benchmark_options.reference b/tests/queries/0_stateless/03203_client_benchmark_options.reference new file mode 100644 index 00000000000..fd2996b1c78 --- /dev/null +++ b/tests/queries/0_stateless/03203_client_benchmark_options.reference @@ -0,0 +1,6 @@ +Ok +Ok +Ok +Ok +Ok +Ok diff --git a/tests/queries/0_stateless/03203_client_benchmark_options.sh b/tests/queries/0_stateless/03203_client_benchmark_options.sh new file mode 100755 index 00000000000..a9b9d69822b --- /dev/null +++ b/tests/queries/0_stateless/03203_client_benchmark_options.sh @@ -0,0 +1,13 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -t -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1 | grep -q "^2\." && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --time -q "SELECT sleepEachRow(2) FORMAT Null" 2>&1 | grep -q "^2\." && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --memory-usage -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9]\+$" && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --memory-usage=none -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" # expected no output +${CLICKHOUSE_CLIENT} --memory-usage=default -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9]\+$" && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --memory-usage=readable -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "^[0-9].*B$" && echo "Ok" || echo "Fail" +${CLICKHOUSE_CLIENT} --memory-usage=unknown -q "SELECT sum(number) FROM numbers(10_000) FORMAT Null" 2>&1 | grep -q "BAD_ARGUMENTS" && echo "Ok" || echo "Fail" From fa2270fd59c0de91d45ac50f2064d31f568935e4 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 11 Jul 2024 22:35:39 +0000 Subject: [PATCH 05/32] Fix VALID UNTIL clause resetting after restart --- .../Access/InterpreterCreateUserQuery.cpp | 17 ++++++++++++++- .../integration/test_user_valid_until/test.py | 21 ++++++++++++++++++- 2 files changed, 36 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index 32c51b745c7..a09de7e688a 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -259,7 +259,22 @@ void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreat if (query.auth_data) auth_data = AuthenticationData::fromAST(*query.auth_data, {}, !query.attach); - updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, {}, allow_no_password, allow_plaintext_password, true); + std::optional valid_until; + if (query.valid_until) + { + const String valid_until_str = checkAndGetLiteralArgument(query.valid_until, "valid_until"); + time_t time = 0; + + if (valid_until_str != "infinity") + { + ReadBufferFromString in(valid_until_str); + readDateTimeText(time, in); + } + + valid_until = time; + } + + updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, valid_until, allow_no_password, allow_plaintext_password, true); } void registerInterpreterCreateUserQuery(InterpreterFactory & factory) diff --git a/tests/integration/test_user_valid_until/test.py b/tests/integration/test_user_valid_until/test.py index d6d5bf8b18e..39ca5997067 100644 --- a/tests/integration/test_user_valid_until/test.py +++ b/tests/integration/test_user_valid_until/test.py @@ -5,7 +5,7 @@ from time import sleep from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node") +node = cluster.add_instance("node", stay_alive=True) @pytest.fixture(scope="module") @@ -84,3 +84,22 @@ def test_details(started_cluster): node.query("SHOW CREATE USER user_details_time_only") == f"CREATE USER user_details_time_only VALID UNTIL \\'{until_year}-01-01 22:03:40\\'\n" ) + + +def test_restart(started_cluster): + node.query("CREATE USER user_restart VALID UNTIL '06/11/2010 08:03:20 Z+3'") + + assert ( + node.query("SHOW CREATE USER user_restart") + == "CREATE USER user_restart VALID UNTIL \\'2010-11-06 05:03:20\\'\n" + ) + + node.restart_clickhouse() + + assert ( + node.query("SHOW CREATE USER user_restart") + == "CREATE USER user_restart VALID UNTIL \\'2010-11-06 05:03:20\\'\n" + ) + + error = "Authentication failed" + assert error in node.query_and_get_error("SELECT 1", user="user_restart") From 705134413ffd321a59c1c2c82af11a5edddc962c Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 12 Jul 2024 12:00:39 +0200 Subject: [PATCH 06/32] Update docs/en/interfaces/cli.md --- docs/en/interfaces/cli.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 63b7353d092..849b3d4b486 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -185,7 +185,7 @@ You can pass parameters to `clickhouse-client` (all parameters have a default va - `--format, -f` – Use the specified default format to output the result. - `--vertical, -E` – If specified, use the [Vertical format](../interfaces/formats.md#vertical) by default to output the result. This is the same as `–format=Vertical`. In this format, each value is printed on a separate line, which is helpful when displaying wide tables. - `--time, -t` – If specified, print the query execution time to ‘stderr’ in non-interactive mode. -- `--memory-usage` – If specified, print the query memory usage to ‘stderr’ in non-interactive mode. +- `--memory-usage` – If specified, print memory usage to ‘stderr’ in non-interactive mode]. Possible values: 'none' - do not print memory usage, 'default' - print number of bytes, 'readable' - print memory usage in human-readable format. - `--stacktrace` – If specified, also print the stack trace if an exception occurs. - `--config-file` – The name of the configuration file. - `--secure` – If specified, will connect to server over secure connection (TLS). You might need to configure your CA certificates in the [configuration file](#configuration_files). The available configuration settings are the same as for [server-side TLS configuration](../operations/server-configuration-parameters/settings.md#server_configuration_parameters-openssl). From 24545caa3a7898e754d7b1f002c95132dc6a2d8b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 15 Jul 2024 00:52:31 +0000 Subject: [PATCH 07/32] Move parsing into a separate function --- .../Access/InterpreterCreateUserQuery.cpp | 59 ++++++++++--------- 1 file changed, 30 insertions(+), 29 deletions(-) diff --git a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp index a09de7e688a..855aa36b159 100644 --- a/src/Interpreters/Access/InterpreterCreateUserQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateUserQuery.cpp @@ -114,6 +114,34 @@ namespace else if (query.grantees) user.grantees = *query.grantees; } + + time_t getValidUntilFromAST(ASTPtr valid_until, ContextPtr context) + { + if (context) + valid_until = evaluateConstantExpressionAsLiteral(valid_until, context); + + const String valid_until_str = checkAndGetLiteralArgument(valid_until, "valid_until"); + + if (valid_until_str == "infinity") + return 0; + + time_t time = 0; + ReadBufferFromString in(valid_until_str); + + if (context) + { + const auto & time_zone = DateLUT::instance(""); + const auto & utc_time_zone = DateLUT::instance("UTC"); + + parseDateTimeBestEffort(time, in, time_zone, utc_time_zone); + } + else + { + readDateTimeText(time, in); + } + + return time; + } } BlockIO InterpreterCreateUserQuery::execute() @@ -134,23 +162,7 @@ BlockIO InterpreterCreateUserQuery::execute() std::optional valid_until; if (query.valid_until) - { - const ASTPtr valid_until_literal = evaluateConstantExpressionAsLiteral(query.valid_until, getContext()); - const String valid_until_str = checkAndGetLiteralArgument(valid_until_literal, "valid_until"); - - time_t time = 0; - - if (valid_until_str != "infinity") - { - const auto & time_zone = DateLUT::instance(""); - const auto & utc_time_zone = DateLUT::instance("UTC"); - - ReadBufferFromString in(valid_until_str); - parseDateTimeBestEffort(time, in, time_zone, utc_time_zone); - } - - valid_until = time; - } + valid_until = getValidUntilFromAST(query.valid_until, getContext()); std::optional default_roles_from_query; if (query.default_roles) @@ -261,18 +273,7 @@ void InterpreterCreateUserQuery::updateUserFromQuery(User & user, const ASTCreat std::optional valid_until; if (query.valid_until) - { - const String valid_until_str = checkAndGetLiteralArgument(query.valid_until, "valid_until"); - time_t time = 0; - - if (valid_until_str != "infinity") - { - ReadBufferFromString in(valid_until_str); - readDateTimeText(time, in); - } - - valid_until = time; - } + valid_until = getValidUntilFromAST(query.valid_until, {}); updateUserFromQueryImpl(user, query, auth_data, {}, {}, {}, {}, valid_until, allow_no_password, allow_plaintext_password, true); } From e346535641bff8998c126f5fb74f02d63014090a Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 15 Jul 2024 16:27:53 +0200 Subject: [PATCH 08/32] fix log in keeper tcp handler --- src/Server/KeeperTCPHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 4849f5827c1..aa79d8814a7 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -623,7 +623,7 @@ void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response LOG_INFO( log, "Total time to process a request took too long ({}ms).\nRequest info: {}", - elapsed, + elapsed_ms, request->toString(/*short_format=*/true)); } From dec5931262ab4dfe541abfca3fb1e7b67f3d9a5e Mon Sep 17 00:00:00 2001 From: Han Fei Date: Mon, 15 Jul 2024 16:43:41 +0200 Subject: [PATCH 09/32] log session id --- src/Server/KeeperTCPHandler.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index aa79d8814a7..b61df45133a 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -622,7 +622,8 @@ void KeeperTCPHandler::updateStats(Coordination::ZooKeeperResponsePtr & response { LOG_INFO( log, - "Total time to process a request took too long ({}ms).\nRequest info: {}", + "Total time to process a request in session {} took too long ({}ms).\nRequest info: {}", + session_id, elapsed_ms, request->toString(/*short_format=*/true)); } From 68aedc219187b0ffe89a3382c7f8136085e72ecf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 03:58:55 +0200 Subject: [PATCH 10/32] Fix something in Fast Test --- docker/test/fasttest/run.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 0d975d64010..58c436b469b 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -9,7 +9,11 @@ trap 'kill $(jobs -pr) ||:' EXIT stage=${stage:-} # Compiler version, normally set by Dockerfile -export LLVM_VERSION=${LLVM_VERSION:-17} +export LLVM_VERSION=${LLVM_VERSION:-18} + +# For some reason it is needed for tests with 'expect' but I don't know, why +groupadd --system --gid 1000 clickhouse +useradd --system --gid 1000 --uid 1000 clickhouse # A variable to pass additional flags to CMake. # Here we explicitly default it to nothing so that bash doesn't complain about From 26f8b81081dd75941bb72b00642391529b9d5bda Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Tue, 16 Jul 2024 10:54:00 +0800 Subject: [PATCH 11/32] update trusted contributors --- tests/ci/lambda_shared_package/lambda_shared/pr.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index e981e28a454..e3aceaab0de 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -45,6 +45,7 @@ TRUSTED_CONTRIBUTORS = { "k-morozov", # Konstantin Morozov, Yandex Cloud "justindeguzman", # ClickHouse, Inc "jrdi", # ClickHouse contributor, TinyBird + "XuJia0210", # ClickHouse, Inc ] } From 919bbf57a7bca538f4a75c0e63af10555b75aa44 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:07:39 +0200 Subject: [PATCH 12/32] Fix hashing of empty tuples --- src/Functions/FunctionsHashing.h | 20 ++++++++++++---- .../03205_hashing_empty_tuples.reference | 22 ++++++++++++++++++ .../03205_hashing_empty_tuples.sql | 23 +++++++++++++++++++ 3 files changed, 60 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/03205_hashing_empty_tuples.reference create mode 100644 tests/queries/0_stateless/03205_hashing_empty_tuples.sql diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 27717ea3611..95c54ac9528 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1184,7 +1184,7 @@ private: if (icolumn->size() != vec_to.size()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument column '{}' size {} doesn't match result column size {} of function {}", - icolumn->getName(), icolumn->size(), vec_to.size(), getName()); + icolumn->getName(), icolumn->size(), vec_to.size(), getName()); if constexpr (Keyed) if (key_cols.size() != vec_to.size() && key_cols.size() != 1) @@ -1223,6 +1223,9 @@ private: else executeGeneric(key_cols, icolumn, vec_to); } + /// Return a fixed random-looking magic number when input is empty. + static constexpr auto filler = 0xe28dbde7fe22e41c; + void executeForArgument(const KeyColumnsType & key_cols, const IDataType * type, const IColumn * column, typename ColumnVector::Container & vec_to, bool & is_first) const { /// Flattening of tuples. @@ -1231,6 +1234,11 @@ private: const auto & tuple_columns = tuple->getColumns(); const DataTypes & tuple_types = typeid_cast(*type).getElements(); size_t tuple_size = tuple_columns.size(); + + if (0 == tuple_size && is_first) + for (auto & hash : vec_to) + hash = static_cast(filler); + for (size_t i = 0; i < tuple_size; ++i) executeForArgument(key_cols, tuple_types[i].get(), tuple_columns[i].get(), vec_to, is_first); } @@ -1239,6 +1247,11 @@ private: const auto & tuple_columns = tuple_const->getColumns(); const DataTypes & tuple_types = typeid_cast(*type).getElements(); size_t tuple_size = tuple_columns.size(); + + if (0 == tuple_size && is_first) + for (auto & hash : vec_to) + hash = static_cast(filler); + for (size_t i = 0; i < tuple_size; ++i) { auto tmp = ColumnConst::create(tuple_columns[i], column->size()); @@ -1300,10 +1313,7 @@ public: constexpr size_t first_data_argument = Keyed; if (arguments.size() <= first_data_argument) - { - /// Return a fixed random-looking magic number when input is empty - vec_to.assign(input_rows_count, static_cast(0xe28dbde7fe22e41c)); - } + vec_to.assign(input_rows_count, static_cast(filler)); KeyColumnsType key_cols{}; if constexpr (Keyed) diff --git a/tests/queries/0_stateless/03205_hashing_empty_tuples.reference b/tests/queries/0_stateless/03205_hashing_empty_tuples.reference new file mode 100644 index 00000000000..e24b5809aee --- /dev/null +++ b/tests/queries/0_stateless/03205_hashing_empty_tuples.reference @@ -0,0 +1,22 @@ +16324913028386710556 +16324913028386710556 +5049034479224883533 +7385293435322750976 +12248912094175844631 +5049034479224883533 +5887129541803688833 +5887129541803688833 +13747979201178469747 +5887129541803688833 +15520217392480966957 +16324913028386710556 +16324913028386710556 +5049034479224883533 +7385293435322750976 +12248912094175844631 +5049034479224883533 +5887129541803688833 +5887129541803688833 +13747979201178469747 +5887129541803688833 +15520217392480966957 diff --git a/tests/queries/0_stateless/03205_hashing_empty_tuples.sql b/tests/queries/0_stateless/03205_hashing_empty_tuples.sql new file mode 100644 index 00000000000..4a97f30ced3 --- /dev/null +++ b/tests/queries/0_stateless/03205_hashing_empty_tuples.sql @@ -0,0 +1,23 @@ +SELECT sipHash64(()); +SELECT sipHash64((), ()); +SELECT sipHash64((), 1); +SELECT sipHash64(1, ()); +SELECT sipHash64(1, (), 1); +SELECT sipHash64((), 1, ()); +SELECT sipHash64((), (1, 2)); +SELECT sipHash64((), (1, 2)); +SELECT sipHash64((1, 2), ()); +SELECT sipHash64((), (1, 2), ()); +SELECT sipHash64((1, 2), (), (3, 4)); + +SELECT sipHash64(materialize(())); +SELECT sipHash64(materialize(()), materialize(())); +SELECT sipHash64(materialize(()), 1); +SELECT sipHash64(1, materialize(())); +SELECT sipHash64(1, materialize(()), 1); +SELECT sipHash64((), 1, materialize(())); +SELECT sipHash64(materialize(()), (1, 2)); +SELECT sipHash64(materialize(()), (1, 2)); +SELECT sipHash64((1, 2), materialize(())); +SELECT sipHash64(materialize(()), (1, 2), ()); +SELECT sipHash64((1, 2), materialize(()), (3, 4)); From 2c62a6b5283c68aac604a97591571e05e6877fab Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:10:26 +0200 Subject: [PATCH 13/32] Update pr.py --- .../lambda_shared_package/lambda_shared/pr.py | 21 +------------------ 1 file changed, 1 insertion(+), 20 deletions(-) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index e3aceaab0de..d970662d38b 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -3,48 +3,29 @@ import re from typing import Tuple -# Individual trusted contirbutors who are not in any trusted organization. +# Individual trusted contributors who are not in any trusted organization. # Can be changed in runtime: we will append users that we learned to be in # a trusted org, to save GitHub API calls. TRUSTED_CONTRIBUTORS = { e.lower() for e in [ - "achimbab", # Kakao corp "Algunenano", # Raúl Marín, ClickHouse, Inc "amosbird", "azat", # SEMRush "bharatnc", # Many contributions. - "bobrik", # Seasoned contributor, CloudFlare "cwurm", # ClickHouse, Inc "den-crane", # Documentation contributor - "hagen1778", # Roman Khavronenko, seasoned contributor - "hczhcz", - "hexiaoting", # Seasoned contributor "ildus", # adjust, ex-pgpro - "javisantana", # a Spanish ClickHouse enthusiast, ex-Carto - "kreuzerkrieg", - "nikvas0", "nvartolomei", # Seasoned contributor, CloudFlare - "spongedu", # Seasoned contributor "taiyang-li", "ucasFL", # Amos Bird's friend "vdimir", # ClickHouse, Inc - "YiuRULE", - "zlobober", # Developer of YT - "ilejn", # Arenadata, responsible for Kerberized Kafka "thomoco", # ClickHouse, Inc - "BoloniniD", # Seasoned contributor, HSE "tonickkozlov", # Cloudflare "tylerhannan", # ClickHouse, Inc - "myrrc", # Mike Kot, DoubleCloud "thevar1able", # ClickHouse, Inc - "aalexfvk", - "MikhailBurdukov", "tsolodov", # ClickHouse, Inc - "kitaisreal", - "k-morozov", # Konstantin Morozov, Yandex Cloud "justindeguzman", # ClickHouse, Inc - "jrdi", # ClickHouse contributor, TinyBird "XuJia0210", # ClickHouse, Inc ] } From 5d96b684a640ef7cc0f911d25ab8fbde4db9f590 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Tue, 16 Jul 2024 11:30:13 +0800 Subject: [PATCH 14/32] update format --- tests/ci/lambda_shared_package/lambda_shared/pr.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index d970662d38b..9320cc4ce8c 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -26,7 +26,7 @@ TRUSTED_CONTRIBUTORS = { "thevar1able", # ClickHouse, Inc "tsolodov", # ClickHouse, Inc "justindeguzman", # ClickHouse, Inc - "XuJia0210", # ClickHouse, Inc + "XuJia0210", # ClickHouse, Inc ] } From 1d2f3a1ed5274764a2a051994ebd69464b6b1212 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 05:56:26 +0200 Subject: [PATCH 15/32] Update pr.py --- tests/ci/lambda_shared_package/lambda_shared/pr.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/ci/lambda_shared_package/lambda_shared/pr.py b/tests/ci/lambda_shared_package/lambda_shared/pr.py index 9320cc4ce8c..95130fc2a0f 100644 --- a/tests/ci/lambda_shared_package/lambda_shared/pr.py +++ b/tests/ci/lambda_shared_package/lambda_shared/pr.py @@ -9,7 +9,6 @@ from typing import Tuple TRUSTED_CONTRIBUTORS = { e.lower() for e in [ - "Algunenano", # Raúl Marín, ClickHouse, Inc "amosbird", "azat", # SEMRush "bharatnc", # Many contributions. @@ -19,11 +18,9 @@ TRUSTED_CONTRIBUTORS = { "nvartolomei", # Seasoned contributor, CloudFlare "taiyang-li", "ucasFL", # Amos Bird's friend - "vdimir", # ClickHouse, Inc "thomoco", # ClickHouse, Inc "tonickkozlov", # Cloudflare "tylerhannan", # ClickHouse, Inc - "thevar1able", # ClickHouse, Inc "tsolodov", # ClickHouse, Inc "justindeguzman", # ClickHouse, Inc "XuJia0210", # ClickHouse, Inc From e292358ade930e2779792365407fa229cd37d079 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 16 Jul 2024 06:12:42 +0200 Subject: [PATCH 16/32] Update run.sh --- docker/test/fasttest/run.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 58c436b469b..57e7b6b2f56 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -11,9 +11,9 @@ stage=${stage:-} # Compiler version, normally set by Dockerfile export LLVM_VERSION=${LLVM_VERSION:-18} -# For some reason it is needed for tests with 'expect' but I don't know, why -groupadd --system --gid 1000 clickhouse -useradd --system --gid 1000 --uid 1000 clickhouse +# For some reason, it is needed for tests with 'expect', but I don't know, why. +sudo groupadd --system --gid 1000 clickhouse +sudo useradd --system --gid 1000 --uid 1000 clickhouse # A variable to pass additional flags to CMake. # Here we explicitly default it to nothing so that bash doesn't complain about From a9b8c2a29a69d85be827048f02ca85a84ec7afca Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 16 Jul 2024 10:48:43 +0200 Subject: [PATCH 17/32] CI: Do not block Tests_3 unless MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI --- tests/ci/functional_test_check.py | 28 +++++++++++- tests/ci/integration_test_check.py | 19 ++++++++ tests/ci/merge_pr.py | 70 ++++-------------------------- 3 files changed, 54 insertions(+), 63 deletions(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 41c7ed963c9..74dd4d8fbd7 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -17,9 +17,19 @@ from download_release_packages import download_last_release from env_helper import REPO_COPY, REPORT_PATH, TEMP_PATH from get_robot_token import get_parameter_from_ssm from pr_info import PRInfo -from report import ERROR, SUCCESS, JobReport, StatusType, TestResults, read_test_results +from report import ( + ERROR, + SUCCESS, + JobReport, + StatusType, + TestResults, + read_test_results, + FAILURE, +) from stopwatch import Stopwatch from tee_popen import TeePopen +from ci_config import CI +from ci_utils import Utils NO_CHANGES_MSG = "Nothing to run" @@ -351,7 +361,23 @@ def main(): additional_files=additional_logs, ).dump(to_file=args.report_to_file if args.report_to_file else None) + should_block_ci = False if state != SUCCESS: + should_block_ci = True + + if state == FAILURE and CI.is_required(check_name): + failed_cnt = Utils.get_failed_tests_number(description) + print( + f"Job status is [{state}] with [{failed_cnt}] failed test cases. status description [{description}]" + ) + if ( + failed_cnt + and failed_cnt <= CI.MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI + ): + print(f"Won't block the CI workflow") + should_block_ci = False + + if should_block_ci: sys.exit(1) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 80ac1935d95..6245f0490fc 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -23,10 +23,13 @@ from report import ( TestResult, TestResults, read_test_results, + FAILURE, ) from stopwatch import Stopwatch import integration_tests_runner as runner +from ci_config import CI +from ci_utils import Utils def get_json_params_dict( @@ -233,7 +236,23 @@ def main(): additional_files=additional_logs, ).dump(to_file=args.report_to_file if args.report_to_file else None) + should_block_ci = False if state != SUCCESS: + should_block_ci = True + + if state == FAILURE and CI.is_required(check_name): + failed_cnt = Utils.get_failed_tests_number(description) + print( + f"Job status is [{state}] with [{failed_cnt}] failed test cases. status description [{description}]" + ) + if ( + failed_cnt + and failed_cnt <= CI.MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI + ): + print(f"Won't block the CI workflow") + should_block_ci = False + + if should_block_ci: sys.exit(1) diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 94456506879..4b57467ebdc 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -27,7 +27,6 @@ from report import SUCCESS, FAILURE from env_helper import GITHUB_UPSTREAM_REPOSITORY, GITHUB_REPOSITORY from synchronizer_utils import SYNC_BRANCH_PREFIX from ci_config import CI -from ci_utils import Utils # The team name for accepted approvals TEAM_NAME = getenv("GITHUB_TEAM_NAME", "core") @@ -249,74 +248,21 @@ def main(): repo = gh.get_repo(args.repo) if args.set_ci_status: + # set Mergeable check status and exit assert args.wf_status in (FAILURE, SUCCESS) - # set mergeable check status and exit commit = get_commit(gh, args.pr_info.sha) statuses = get_commit_filtered_statuses(commit) - max_failed_tests_per_job = 0 - job_name_with_max_failures = None - total_failed_tests = 0 - failed_to_get_info = False has_failed_statuses = False for status in statuses: - if not CI.is_required(status.context) or status.context in ( - CI.StatusNames.SYNC, - CI.StatusNames.PR_CHECK, - ): - # CI.StatusNames.SYNC or CI.StatusNames.PR_CHECK should not be checked - continue print(f"Check status [{status.context}], [{status.state}]") - if status.state == FAILURE: + if not CI.is_required(status.context) and status.state != SUCCESS: has_failed_statuses = True - failed_cnt = Utils.get_failed_tests_number(status.description) - if failed_cnt is None: - failed_to_get_info = True - print( - f"WARNING: failed to get number of failed tests from [{status.description}]" - ) - else: - if failed_cnt > max_failed_tests_per_job: - job_name_with_max_failures = status.context - max_failed_tests_per_job = failed_cnt - total_failed_tests += failed_cnt - print( - f"Failed test cases in [{status.context}] is [{failed_cnt}], total failures [{total_failed_tests}]" - ) - elif status.state != SUCCESS and status.context not in ( - CI.StatusNames.SYNC, - CI.StatusNames.PR_CHECK, - ): - # do not block CI on failures in (CI.StatusNames.SYNC, CI.StatusNames.PR_CHECK) - has_failed_statuses = True - print( - f"Unexpected status for [{status.context}]: [{status.state}] - block further testing" - ) - failed_to_get_info = True - - can_continue = True - if total_failed_tests > CI.MAX_TOTAL_FAILURES_BEFORE_BLOCKING_CI: - print( - f"Required check has [{total_failed_tests}] failed - block further testing" - ) - can_continue = False - if max_failed_tests_per_job > CI.MAX_TOTAL_FAILURES_PER_JOB_BEFORE_BLOCKING_CI: - print( - f"Job [{job_name_with_max_failures}] has [{max_failed_tests_per_job}] failures - block further testing" - ) - can_continue = False - if failed_to_get_info: - print("Unexpected commit status state - block further testing") - can_continue = False - if args.wf_status != SUCCESS and not has_failed_statuses: - # workflow failed but reason is unknown as no failed statuses present - can_continue = False - print( - "WARNING: Either the runner is faulty or the operating status is unknown. The first is self-healing, the second requires investigation." - ) if args.wf_status == SUCCESS or has_failed_statuses: - # do not set mergeable check status if args.wf_status == failure, apparently it has died runners and is to be restarted + # set Mergeable check if workflow is successful (green) + # or if we have GH statuses with failures (red) + # to avoid false-green on a died runner state = trigger_mergeable_check( commit, statuses, @@ -333,10 +279,10 @@ def main(): print( "Workflow failed but no failed statuses found (died runner?) - cannot set Mergeable Check status" ) - - if not can_continue: + if args.wf_status == SUCCESS and not has_failed_statuses: + sys.exit(0) + else: sys.exit(1) - sys.exit(0) # An ugly and not nice fix to patch the wrong organization URL, # see https://github.com/PyGithub/PyGithub/issues/2395#issuecomment-1378629710 From 2af361eb14617d067732795424c22a6f511c9f18 Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 16 Jul 2024 16:18:41 +0200 Subject: [PATCH 18/32] minor fix --- tests/ci/merge_pr.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index 4b57467ebdc..b005be472c7 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -256,7 +256,12 @@ def main(): has_failed_statuses = False for status in statuses: print(f"Check status [{status.context}], [{status.state}]") - if not CI.is_required(status.context) and status.state != SUCCESS: + if ( + not CI.is_required(status.context) + and status.context not in (CI.StatusNames.SYNC, CI.StatusNames.PR_CHECK) + and status.state != SUCCESS + ): + print(f"WARNING: Failed status [{status.context}], [{status.state}]") has_failed_statuses = True if args.wf_status == SUCCESS or has_failed_statuses: From db7fb78a04280179b7ca09f588c957d2e0eab6be Mon Sep 17 00:00:00 2001 From: Max K Date: Tue, 16 Jul 2024 19:49:21 +0200 Subject: [PATCH 19/32] major fix --- tests/ci/ci_config.py | 6 ++++-- tests/ci/merge_pr.py | 6 +----- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 9b9ddee5326..f5e39f343b2 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -415,7 +415,8 @@ class CI: JobNames.INTEGRATION_TEST_FLAKY: CommonJobConfigs.INTEGRATION_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True, - reference_job_name=JobNames.INTEGRATION_TEST_TSAN, + # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf + # reference_job_name=JobNames.INTEGRATION_TEST_TSAN, ), JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], @@ -460,7 +461,8 @@ class CI: required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True, timeout=3600, - reference_job_name=JobNames.STATELESS_TEST_RELEASE, + # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf + # reference_job_name=JobNames.STATELESS_TEST_RELEASE, ), JobNames.JEPSEN_KEEPER: JobConfig( required_builds=[BuildNames.BINARY_RELEASE], diff --git a/tests/ci/merge_pr.py b/tests/ci/merge_pr.py index b005be472c7..4d8facafb84 100644 --- a/tests/ci/merge_pr.py +++ b/tests/ci/merge_pr.py @@ -256,11 +256,7 @@ def main(): has_failed_statuses = False for status in statuses: print(f"Check status [{status.context}], [{status.state}]") - if ( - not CI.is_required(status.context) - and status.context not in (CI.StatusNames.SYNC, CI.StatusNames.PR_CHECK) - and status.state != SUCCESS - ): + if CI.is_required(status.context) and status.state != SUCCESS: print(f"WARNING: Failed status [{status.context}], [{status.state}]") has_failed_statuses = True From ea3a333a0711a33ad9193e67939b5f8a970834ea Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Tue, 16 Jul 2024 22:25:45 +0200 Subject: [PATCH 20/32] Fix formatting error --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 260457b3be1..5e2802eb08b 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -86,7 +86,7 @@ Returns the fully qualified domain name of the ClickHouse server. fqdn(); ``` -Aliases: `fullHostName`, 'FQDN'. +Aliases: `fullHostName`, `FQDN`. **Returned value** From 8c0ade413cca3ea63b47909c54c80de76c1e2a26 Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Tue, 16 Jul 2024 22:28:12 +0200 Subject: [PATCH 21/32] Update other-functions.md --- docs/en/sql-reference/functions/other-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 5e2802eb08b..b7e4094f30e 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -76,7 +76,7 @@ WHERE macro = 'test'; └───────┴──────────────┘ ``` -## FQDN +## fqdn Returns the fully qualified domain name of the ClickHouse server. From 5ca7241865f863d68b3a3b96011b9238e49ac3f8 Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 17 Jul 2024 00:23:14 +0200 Subject: [PATCH 22/32] CI: AWS ASG decrease capacity from runners --- tests/ci/worker/init_runner.sh | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index ec7793db2aa..91bffa876b2 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -50,7 +50,7 @@ set -uo pipefail # set accordingly to a runner role # #################################### -echo "Running init script" +echo "Running init script v1" export DEBIAN_FRONTEND=noninteractive export RUNNER_HOME=/home/ubuntu/actions-runner @@ -90,7 +90,6 @@ terminate_delayed() { # IF `sleep` IS CHANGED, CHANGE ANOTHER VALUE IN `pgrep` sleep=13.14159265358979323846 echo "Going to terminate the runner's instance in $sleep seconds" - INSTANCE_ID=$(ec2metadata --instance-id) # We execute it with `at` to not have it as an orphan process, but launched independently # GH Runners kill all remain processes echo "sleep '$sleep'; aws ec2 terminate-instances --instance-ids $INSTANCE_ID" | at now || \ @@ -111,11 +110,17 @@ declare -f terminate_delayed >> /tmp/actions-hooks/common.sh terminate_and_exit() { # Terminate instance and exit from the script instantly echo "Going to terminate the runner's instance" - INSTANCE_ID=$(ec2metadata --instance-id) aws ec2 terminate-instances --instance-ids "$INSTANCE_ID" exit 0 } +terminate_decrease_and_exit() { + # Terminate instance and exit from the script instantly + echo "Going to terminate the runner's instance and decrease asg capacity" + aws autoscaling terminate-instance-in-auto-scaling-group --instance-id "$INSTANCE_ID" --should-decrement-desired-capacity + exit 0 +} + declare -f terminate_and_exit >> /tmp/actions-hooks/common.sh check_spot_instance_is_old() { @@ -324,7 +329,7 @@ while true; do sudo -u ubuntu ./config.sh remove --token "$(get_runner_token)" \ || continue echo "Runner didn't launch or have assigned jobs after ${RUNNER_AGE} seconds, shutting down" - terminate_and_exit + terminate_decrease_and_exit fi fi else From a1a802cee9f1e75cfdaed7c86aac63fd4e3010db Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 17 Jul 2024 00:32:13 +0200 Subject: [PATCH 23/32] fix deployment script --- tests/ci/worker/deploy-runner-init.sh | 4 +++- tests/ci/worker/init_runner.sh | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/ci/worker/deploy-runner-init.sh b/tests/ci/worker/deploy-runner-init.sh index 06edede48fa..96fbd82a99c 100755 --- a/tests/ci/worker/deploy-runner-init.sh +++ b/tests/ci/worker/deploy-runner-init.sh @@ -1,5 +1,7 @@ #!/usr/bin/env bash +set -e + usage() { echo "Usage: $0 ENVIRONMENT" >&2 echo "Valid values for ENVIRONMENT: staging, production" >&2 @@ -55,7 +57,7 @@ EOF body() { local first_line - first_line=$(sed -n '/^# THE SCRIPT START$/{=;q}' "$SOURCE_SCRIPT") + first_line=$(sed -n '/^# THE SCRIPT START$/{=;q;}' "$SOURCE_SCRIPT") if [ -z "$first_line" ]; then echo "The pattern '# THE SCRIPT START' is not found in $SOURCE_SCRIPT" >&2 exit 1 diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index 91bffa876b2..d6cdb6d9c57 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -50,7 +50,7 @@ set -uo pipefail # set accordingly to a runner role # #################################### -echo "Running init script v1" +echo "Running init v1" export DEBIAN_FRONTEND=noninteractive export RUNNER_HOME=/home/ubuntu/actions-runner From 0d1789292bc480c33c0f2b2ee54058976dea74fe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 02:11:09 +0200 Subject: [PATCH 24/32] Add sudo for convenience --- docker/test/util/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/util/Dockerfile b/docker/test/util/Dockerfile index f13bb576f79..babddbd228c 100644 --- a/docker/test/util/Dockerfile +++ b/docker/test/util/Dockerfile @@ -17,6 +17,7 @@ ENV DEBIAN_FRONTEND=noninteractive LLVM_VERSION=18 RUN apt-get update \ && apt-get install \ + sudo \ apt-transport-https \ apt-utils \ ca-certificates \ From c913a36ed6580e87ebb641cb3024c7182725d8ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 02:13:57 +0200 Subject: [PATCH 25/32] Minor modifications --- docker/test/fasttest/Dockerfile | 5 ++++- docker/test/fasttest/run.sh | 4 ---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 2512268be0f..72e33226f38 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -26,7 +26,10 @@ RUN apt-get update \ zstd \ --yes --no-install-recommends \ && apt-get clean \ - && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* + && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* \ + && groupadd --system --gid 1000 clickhouse \ + && useradd --system --gid 1000 --uid 1000 clickhouse +# ^ For some reason, groupadd and useradd are needed for tests with 'expect', but I don't know, why. COPY requirements.txt / RUN pip3 install --no-cache-dir -r /requirements.txt diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 57e7b6b2f56..970bf12a81a 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -11,10 +11,6 @@ stage=${stage:-} # Compiler version, normally set by Dockerfile export LLVM_VERSION=${LLVM_VERSION:-18} -# For some reason, it is needed for tests with 'expect', but I don't know, why. -sudo groupadd --system --gid 1000 clickhouse -sudo useradd --system --gid 1000 --uid 1000 clickhouse - # A variable to pass additional flags to CMake. # Here we explicitly default it to nothing so that bash doesn't complain about # it being undefined. Also read it as array so that we can pass an empty list From a40edea5c99ac290a3b04a015065c0cad5266ab0 Mon Sep 17 00:00:00 2001 From: Xu Jia Date: Wed, 17 Jul 2024 09:37:28 +0800 Subject: [PATCH 26/32] trigger CI From 60f45cb993ccf15812605d51b6e7ce8489c98d05 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 17 Jul 2024 03:52:35 +0200 Subject: [PATCH 27/32] A fix --- docker/test/fasttest/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile index 72e33226f38..4cac2ee6135 100644 --- a/docker/test/fasttest/Dockerfile +++ b/docker/test/fasttest/Dockerfile @@ -28,7 +28,7 @@ RUN apt-get update \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/* \ && groupadd --system --gid 1000 clickhouse \ - && useradd --system --gid 1000 --uid 1000 clickhouse + && useradd --system --gid 1000 --uid 1000 -m clickhouse # ^ For some reason, groupadd and useradd are needed for tests with 'expect', but I don't know, why. COPY requirements.txt / From 005256429fb4d6a9aec88b3bb5662382166f6f9b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 17 Jul 2024 11:49:15 +0200 Subject: [PATCH 28/32] Don't run test 03198_dynamic_read_subcolumns in azure --- tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql index 27fca179580..155009cc2fe 100644 --- a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql +++ b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql @@ -1,4 +1,4 @@ --- Tags: no-random-settings, no-s3-storage +-- Tags: no-random-settings, no-object-storage SET allow_experimental_dynamic_type = 1; DROP TABLE IF EXISTS test_dynamic; From 2958803255f0f646147eca4f9432f72496318bbf Mon Sep 17 00:00:00 2001 From: Max K Date: Wed, 17 Jul 2024 11:26:38 +0200 Subject: [PATCH 29/32] do not decrease from lambda --- tests/ci/autoscale_runners_lambda/app.py | 77 ++++++++++--------- .../test_autoscale.py | 25 ++++-- 2 files changed, 60 insertions(+), 42 deletions(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index 520abcb41bb..aed5cf3a8dd 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -99,7 +99,8 @@ def set_capacity( continue raise ValueError("Queue status is not in ['in_progress', 'queued']") - scale_down, scale_up = get_scales(runner_type) + #scale_down, scale_up = get_scales(runner_type) + _, scale_up = get_scales(runner_type) # With lyfecycle hooks some instances are actually free because some of # them are in 'Terminating:Wait' state effective_capacity = max( @@ -110,7 +111,7 @@ def set_capacity( # How much nodes are free (positive) or need to be added (negative) capacity_reserve = effective_capacity - running - queued stop = False - if capacity_reserve < 0: + if capacity_reserve <= 0: # This part is about scaling up capacity_deficit = -capacity_reserve # It looks that we are still OK, since no queued jobs exist @@ -158,41 +159,43 @@ def set_capacity( ) return - # Now we will calculate if we need to scale down - stop = stop or asg["DesiredCapacity"] == asg["MinSize"] - new_capacity = asg["DesiredCapacity"] - (capacity_reserve // scale_down) - new_capacity = max(new_capacity, asg["MinSize"]) - new_capacity = min(new_capacity, asg["MaxSize"]) - stop = stop or asg["DesiredCapacity"] == new_capacity - if stop: - logging.info( - "Do not decrease ASG %s capacity, current capacity=%s, effective " - "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", - asg["AutoScalingGroupName"], - asg["DesiredCapacity"], - effective_capacity, - asg["MinSize"], - running, - queued, - ) - return - - logging.info( - "The ASG %s capacity will be decreased to %s, current capacity=%s, effective " - "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", - asg["AutoScalingGroupName"], - new_capacity, - asg["DesiredCapacity"], - effective_capacity, - asg["MinSize"], - running, - queued, - ) - if not dry_run: - client.set_desired_capacity( - AutoScalingGroupName=asg["AutoScalingGroupName"], - DesiredCapacity=new_capacity, - ) + # FIXME: try decreasing capacity from runners that finished their jobs and have no job assigned + # IMPORTANT: Runner init script must be of version that supports ASG decrease + # # Now we will calculate if we need to scale down + # stop = stop or asg["DesiredCapacity"] == asg["MinSize"] + # new_capacity = asg["DesiredCapacity"] - (capacity_reserve // scale_down) + # new_capacity = max(new_capacity, asg["MinSize"]) + # new_capacity = min(new_capacity, asg["MaxSize"]) + # stop = stop or asg["DesiredCapacity"] == new_capacity + # if stop: + # logging.info( + # "Do not decrease ASG %s capacity, current capacity=%s, effective " + # "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", + # asg["AutoScalingGroupName"], + # asg["DesiredCapacity"], + # effective_capacity, + # asg["MinSize"], + # running, + # queued, + # ) + # return + # + # logging.info( + # "The ASG %s capacity will be decreased to %s, current capacity=%s, effective " + # "capacity=%s, minimum capacity=%s, running jobs=%s, queue size=%s", + # asg["AutoScalingGroupName"], + # new_capacity, + # asg["DesiredCapacity"], + # effective_capacity, + # asg["MinSize"], + # running, + # queued, + # ) + # if not dry_run: + # client.set_desired_capacity( + # AutoScalingGroupName=asg["AutoScalingGroupName"], + # DesiredCapacity=new_capacity, + # ) def main(dry_run: bool = True) -> None: diff --git a/tests/ci/autoscale_runners_lambda/test_autoscale.py b/tests/ci/autoscale_runners_lambda/test_autoscale.py index 3aeab49ffc7..d1a1f9b358f 100644 --- a/tests/ci/autoscale_runners_lambda/test_autoscale.py +++ b/tests/ci/autoscale_runners_lambda/test_autoscale.py @@ -97,19 +97,34 @@ class TestSetCapacity(unittest.TestCase): ), TestCase("lower-min", 10, 5, 20, [Queue("queued", 5, "lower-min")], 10), # Decrease capacity - TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], 5), + # FIXME: Tests changed for lambda that can only scale up + # TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], 5), + TestCase("w/reserve", 1, 13, 20, [Queue("queued", 5, "w/reserve")], -1), + # TestCase( + # "style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], 5 + # ), TestCase( - "style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], 5 + "style-checker", 1, 13, 20, [Queue("queued", 5, "style-checker")], -1 ), - TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], 17), - TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], 3), + # TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], 17), + TestCase("w/reserve", 1, 23, 20, [Queue("queued", 17, "w/reserve")], -1), + # TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], 3), + TestCase("decrease", 1, 13, 20, [Queue("in_progress", 3, "decrease")], -1), + # TestCase( + # "style-checker", + # 1, + # 13, + # 20, + # [Queue("in_progress", 5, "style-checker")], + # 5, + # ), TestCase( "style-checker", 1, 13, 20, [Queue("in_progress", 5, "style-checker")], - 5, + -1, ), ) for t in test_cases: From c40473ff80c0cdf09b85b7dda695a41ee2df0112 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 17 Jul 2024 10:50:52 +0000 Subject: [PATCH 30/32] Automatic style fix --- tests/ci/autoscale_runners_lambda/app.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/autoscale_runners_lambda/app.py b/tests/ci/autoscale_runners_lambda/app.py index aed5cf3a8dd..c790a5081dd 100644 --- a/tests/ci/autoscale_runners_lambda/app.py +++ b/tests/ci/autoscale_runners_lambda/app.py @@ -99,7 +99,7 @@ def set_capacity( continue raise ValueError("Queue status is not in ['in_progress', 'queued']") - #scale_down, scale_up = get_scales(runner_type) + # scale_down, scale_up = get_scales(runner_type) _, scale_up = get_scales(runner_type) # With lyfecycle hooks some instances are actually free because some of # them are in 'Terminating:Wait' state From 3ade6de95df356adfaf7f6a6e3fb22d2545c6c50 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 17 Jul 2024 13:13:51 +0200 Subject: [PATCH 31/32] Add description about no-object-storage tag --- tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql index 155009cc2fe..5d473064c68 100644 --- a/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql +++ b/tests/queries/0_stateless/03198_dynamic_read_subcolumns.sql @@ -1,4 +1,5 @@ -- Tags: no-random-settings, no-object-storage +-- Tag no-object-storage: this test relies on the number of opened files in MergeTree that can differ in object storages SET allow_experimental_dynamic_type = 1; DROP TABLE IF EXISTS test_dynamic; From 245bc587ae5d9cb6751153b7bf24dcf88487d5dd Mon Sep 17 00:00:00 2001 From: Shaun Struwig <41984034+Blargian@users.noreply.github.com> Date: Wed, 17 Jul 2024 17:57:28 +0200 Subject: [PATCH 32/32] add 'fqdn' to list --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 4e7bf7f686e..b4b18f9bbf6 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1653,6 +1653,7 @@ formated formatschema formatter formatters +fqdn frac freezed fromDaysSinceYearZero