From 48a9d2b660fb4d8ece47e87ce02df66ab6a8e897 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Thu, 12 Sep 2024 08:36:55 +0000 Subject: [PATCH 01/65] Add enable_secure_identifiers setting to disallow insecure identifiers --- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/Interpreters/executeQuery.cpp | 8 ++ src/Parsers/IAST.cpp | 16 +++ src/Parsers/IAST.h | 7 +- .../03234_enable_secure_identifiers.reference | 3 + .../03234_enable_secure_identifiers.sql | 111 ++++++++++++++++++ 7 files changed, 146 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03234_enable_secure_identifiers.reference create mode 100644 tests/queries/0_stateless/03234_enable_secure_identifiers.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 493752fc3fe..a5a7020c48d 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -901,6 +901,7 @@ class IColumn; M(Bool, restore_replace_external_engines_to_null, false, "Replace all the external table engines to Null on restore. Useful for testing purposes", 0) \ M(Bool, restore_replace_external_table_functions_to_null, false, "Replace all table functions to Null on restore. Useful for testing purposes", 0) \ M(Bool, create_if_not_exists, false, "Enable IF NOT EXISTS for CREATE statements by default", 0) \ + M(Bool, enable_secure_identifiers, false, "If enabled, only secure identifiers are allowed", 0) \ \ \ /* ###################################### */ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 19f2d5ccdf0..ec9e6d0e8e8 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -79,6 +79,7 @@ static std::initializer_list executeQueryImpl( InterpreterSetQuery::applySettingsFromQuery(ast, context); validateAnalyzerSettings(ast, context->getSettingsRef().allow_experimental_analyzer); + if (settings.enable_secure_identifiers) + { + WriteBufferFromOwnString buf; + IAST::FormatSettings enable_secure_identifiers_settings(buf, true); + enable_secure_identifiers_settings.enable_secure_identifiers = true; + ast->format(enable_secure_identifiers_settings); + } + if (auto * insert_query = ast->as()) insert_query->tail = istr; diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 54b11f2888e..0ea64ccfdbd 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -219,6 +219,7 @@ String IAST::getColumnNameWithoutAlias() const void IAST::FormatSettings::writeIdentifier(const String & name) const { + checkIdentifier(name); switch (identifier_quoting_style) { case IdentifierQuotingStyle::None: @@ -260,6 +261,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const void IAST::FormatSettings::quoteIdentifier(const String & name) const { + checkIdentifier(name); switch (identifier_quoting_style) { case IdentifierQuotingStyle::None: @@ -285,6 +287,20 @@ void IAST::FormatSettings::quoteIdentifier(const String & name) const } } +void IAST::FormatSettings::checkIdentifier(const String & name) const +{ + if (enable_secure_identifiers) + { + for (char c : name) + { + if (!std::isalnum(c) && c != '_') + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not a secure identifier: `{}`", name); + } + } + } +} + void IAST::dumpTree(WriteBuffer & ostr, size_t indent) const { String indent_str(indent, '-'); diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index dfb6a6cbeba..e6a453ac4d7 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -202,6 +202,7 @@ public: char nl_or_ws; /// Newline or whitespace. LiteralEscapingStyle literal_escaping_style; bool print_pretty_type_names; + bool enable_secure_identifiers; explicit FormatSettings( WriteBuffer & ostr_, @@ -211,7 +212,8 @@ public: IdentifierQuotingStyle identifier_quoting_style_ = IdentifierQuotingStyle::Backticks, bool show_secrets_ = true, LiteralEscapingStyle literal_escaping_style_ = LiteralEscapingStyle::Regular, - bool print_pretty_type_names_ = false) + bool print_pretty_type_names_ = false, + bool enable_secure_identifiers_ = false) : ostr(ostr_) , one_line(one_line_) , hilite(hilite_) @@ -221,6 +223,7 @@ public: , nl_or_ws(one_line ? ' ' : '\n') , literal_escaping_style(literal_escaping_style_) , print_pretty_type_names(print_pretty_type_names_) + , enable_secure_identifiers(enable_secure_identifiers_) { } @@ -234,6 +237,7 @@ public: , nl_or_ws(other.nl_or_ws) , literal_escaping_style(other.literal_escaping_style) , print_pretty_type_names(other.print_pretty_type_names) + , enable_secure_identifiers(other.enable_secure_identifiers) { } @@ -241,6 +245,7 @@ public: // Quote identifier `name` even when `always_quote_identifiers` is false. // If `identifier_quoting_style` is `IdentifierQuotingStyle::None`, quote it with `IdentifierQuotingStyle::Backticks` void quoteIdentifier(const String & name) const; + void checkIdentifier(const String & name) const; }; /// State. For example, a set of nodes can be remembered, which we already walk through. diff --git a/tests/queries/0_stateless/03234_enable_secure_identifiers.reference b/tests/queries/0_stateless/03234_enable_secure_identifiers.reference new file mode 100644 index 00000000000..60069e2686e --- /dev/null +++ b/tests/queries/0_stateless/03234_enable_secure_identifiers.reference @@ -0,0 +1,3 @@ +CREATE TABLE default.test_foo\n(\n `secure_123` Int8,\n `date` Date,\n `town` LowCardinality(String)\n)\nENGINE = MergeTree\nPARTITION BY toYear(date)\nPRIMARY KEY (town, date)\nORDER BY (town, date)\nSETTINGS index_granularity = 8192\nCOMMENT \'test\' +CREATE TABLE default.test_foo\n(\n `123_secure` Int8,\n `date` Date,\n `town` LowCardinality(String)\n)\nENGINE = MergeTree\nPARTITION BY toYear(date)\nPRIMARY KEY (town, date)\nORDER BY (town, date)\nSETTINGS index_granularity = 8192\nCOMMENT \'test\' +CREATE TABLE default.test_foo\n(\n `insecure_$` Int8,\n `date` Date,\n `town` LowCardinality(String)\n)\nENGINE = MergeTree\nPARTITION BY toYear(date)\nPRIMARY KEY (town, date)\nORDER BY (town, date)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/03234_enable_secure_identifiers.sql b/tests/queries/0_stateless/03234_enable_secure_identifiers.sql new file mode 100644 index 00000000000..dfeb53dd147 --- /dev/null +++ b/tests/queries/0_stateless/03234_enable_secure_identifiers.sql @@ -0,0 +1,111 @@ +DROP TABLE IF EXISTS `test_foo_#`; +CREATE TABLE `test_foo_#` ( + `date` Date, + `town` LowCardinality(String), +) +ENGINE = MergeTree +PRIMARY KEY (town, date) +PARTITION BY toYear(date) +COMMENT 'test' -- to end ENGINE definition, so SETTINGS will be in the query level +SETTINGS + enable_secure_identifiers=true; -- { serverError BAD_ARGUMENTS } +DROP TABLE IF EXISTS `test_foo_#`; + + +DROP TABLE IF EXISTS test_foo; +CREATE TABLE test_foo ( + `insecure_#` Int8, + `date` Date, + `town` LowCardinality(String), +) +ENGINE = MergeTree +PRIMARY KEY (town, date) +PARTITION BY toYear(date) +COMMENT 'test' -- to end ENGINE definition, so SETTINGS will be in the query level +SETTINGS + enable_secure_identifiers=true; -- { serverError BAD_ARGUMENTS } + +DROP TABLE IF EXISTS test_foo; +CREATE TABLE test_foo ( + `insecure_'` Int8, + `date` Date, + `town` LowCardinality(String), +) +ENGINE = MergeTree +PRIMARY KEY (town, date) +PARTITION BY toYear(date) +COMMENT 'test' -- to end ENGINE definition, so SETTINGS will be in the query level +SETTINGS + enable_secure_identifiers=true; -- { serverError BAD_ARGUMENTS } + +DROP TABLE IF EXISTS test_foo; +CREATE TABLE test_foo ( + `insecure_"` Int8, + `date` Date, + `town` LowCardinality(String), +) +ENGINE = MergeTree +PRIMARY KEY (town, date) +PARTITION BY toYear(date) +COMMENT 'test' -- to end ENGINE definition, so SETTINGS will be in the query level +SETTINGS + enable_secure_identifiers=true; -- { serverError BAD_ARGUMENTS } + +DROP TABLE IF EXISTS test_foo; +CREATE TABLE test_foo ( + `secure_123` Int8, + `date` Date, + `town` LowCardinality(String), +) +ENGINE = MergeTree +PRIMARY KEY (town, date) +PARTITION BY toYear(date) +COMMENT 'test' -- to end ENGINE definition, so SETTINGS will be in the query level +SETTINGS + enable_secure_identifiers=true; + +SHOW CREATE TABLE test_foo +SETTINGS + enable_secure_identifiers=true; + +DROP TABLE IF EXISTS test_foo; +CREATE TABLE test_foo ( + `123_secure` Int8, + `date` Date, + `town` LowCardinality(String), +) +ENGINE = MergeTree +PRIMARY KEY (town, date) +PARTITION BY toYear(date) +COMMENT 'test' -- to end ENGINE definition, so SETTINGS will be in the query level +SETTINGS + enable_secure_identifiers=true; + +SHOW CREATE TABLE test_foo +SETTINGS + enable_secure_identifiers=true; + +-- CREATE TABLE without `enable_secure_identifiers` +DROP TABLE IF EXISTS test_foo; +CREATE TABLE `test_foo` ( + `insecure_$` Int8, + `date` Date, + `town` LowCardinality(String), +) +ENGINE = MergeTree +PRIMARY KEY (town, date) +PARTITION BY toYear(date); +-- Then SHOW CREATE .. with `enable_secure_identifiers` +-- While the result contains insecure identifiers (`insecure_$`), the `SHOW CREATE TABLE ...` query does not have any. So the query is expected to succeed. +SHOW CREATE TABLE test_foo +SETTINGS + enable_secure_identifiers=true; + +DROP TABLE IF EXISTS test_foo; + +-- SHOW CREATE .. query contains an insecure identifier (`test_foo$`) with `enable_secure_identifiers` +SHOW CREATE TABLE `test_foo$` +SETTINGS + enable_secure_identifiers=true; -- { serverError BAD_ARGUMENTS } + +DROP TABLE IF EXISTS test_foo; \ No newline at end of file From 707210ed38a9d9d1f2338bced1d17277e4841800 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Wed, 18 Sep 2024 16:25:22 +0000 Subject: [PATCH 02/65] Added new settings min_free_disk_bytes_to_throw_insert and min_free_disk_ratio_to_throw_insert to stop inserts when disk is near full. --- .../settings/merge-tree-settings.md | 26 +++++++- src/Core/Defines.h | 1 + src/Core/Settings.h | 2 + src/Core/SettingsChangesHistory.cpp | 4 +- src/Core/SettingsQuirks.cpp | 8 +++ .../MergeTree/MergeTreeDataWriter.cpp | 36 ++++++++++- src/Storages/MergeTree/MergeTreeSettings.cpp | 8 +++ src/Storages/MergeTree/MergeTreeSettings.h | 2 + .../config.d/storage_configuration.xml | 19 ++++++ .../test.py | 61 +++++++++++++++++++ 10 files changed, 163 insertions(+), 4 deletions(-) create mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml create mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/test.py diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index a13aacc76e6..af8e16c30ea 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -1064,4 +1064,28 @@ Possible values: - throw, drop, rebuild -Default value: throw \ No newline at end of file +Default value: throw + +## min_free_disk_bytes_to_throw_insert + +The minimum number of bytes that should be free in disk space in order to insert data. If the number of available free bytes is less than `min_free_disk_bytes_to_throw_insert` then an exception is thrown and the insert is not executed. Note that this setting: +- takes into account the `keep_free_space_bytes` setting. +- does not take into account the amount of data that will be written by the `INSERT` operation. + +Possible values: + +- Any positive integer. + +Default value: 0 bytes. + +## min_free_disk_ratio_to_throw_insert + +The minimum free to total disk space ratio to perform an `INSERT`. Must be a floating point value between 0 and 1. Note that this setting: +- takes into account the `keep_free_space_bytes` setting. +- does not take into account the amount of data that will be written by the `INSERT` operation. + +Possible values: + +- Float, 0.0 - 1.0 + +Default value: 0.0 \ No newline at end of file diff --git a/src/Core/Defines.h b/src/Core/Defines.h index c6e65f34e90..81195af6b31 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -108,6 +108,7 @@ static constexpr auto DEFAULT_QUERY_CACHE_MAX_SIZE = 1_GiB; static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRIES = 1024uz; static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_BYTES = 1_MiB; static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS = 30'000'000uz; +static constexpr Float64 DEFAULT_MIN_FREE_DISK_RATIO = 0.0; /// Query profiler cannot work with sanitizers. /// Sanitizers are using quick "frame walking" stack unwinding (this implies -fno-omit-frame-pointer) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index fadc4079fe0..95f585d1a27 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -343,6 +343,8 @@ class IColumn; M(Int64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite. Zero means async mode.", 0) \ M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ + M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ + M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ \ M(Bool, final, false, "Query with the FINAL modifier by default. If the engine does not support final, it does not have any effect. On queries with multiple tables final is applied only on those that support it. It also works on distributed tables", 0) \ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 560f144866b..da0fbfad255 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -85,7 +85,9 @@ static std::initializer_list(); + min_free_disk_ratio_to_throw_insert < 0.0 || min_free_disk_ratio_to_throw_insert > 1) + { + if (log) + LOG_WARNING(log, "Sanity check: 'min_free_disk_ratio_to_throw_insert' must be between 0.0 and 1.0. Set to default (0.0)"); + current_settings.set("min_free_disk_ratio_to_throw_insert", DEFAULT_MIN_FREE_DISK_RATIO); + } } } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index f29d715e791..acccfeb6c30 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -60,6 +60,7 @@ namespace ErrorCodes extern const int ABORTED; extern const int LOGICAL_ERROR; extern const int TOO_MANY_PARTS; + extern const int NOT_ENOUGH_SPACE; } namespace @@ -553,6 +554,39 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( VolumePtr volume = data.getStoragePolicy()->getVolume(0); VolumePtr data_part_volume = createVolumeFromReservation(reservation, volume); + const auto & global_settings = context->getSettingsRef(); + const auto & data_settings = data.getSettings(); + + UInt64 min_bytes = global_settings.min_free_disk_bytes_to_throw_insert; + if (data_settings->min_free_disk_bytes_to_throw_insert.changed) + min_bytes = data_settings->min_free_disk_bytes_to_throw_insert; + + Float64 min_ratio = global_settings.min_free_disk_ratio_to_throw_insert; + if (data_settings->min_free_disk_ratio_to_throw_insert.changed) + min_ratio = data_settings->min_free_disk_ratio_to_throw_insert; + + if (min_bytes > 0 || min_ratio > 0.0) + { + const auto disk = data_part_volume->getDisk(); + const UInt64 total_disk_bytes = *disk->getTotalSpace(); + const UInt64 free_disk_bytes = *disk->getAvailableSpace(); + + const UInt64 min_bytes_from_ratio = static_cast(min_ratio * total_disk_bytes); + const UInt64 needed_free_bytes = std::max(min_bytes, min_bytes_from_ratio); + + if (needed_free_bytes > free_disk_bytes) + { + throw Exception( + ErrorCodes::NOT_ENOUGH_SPACE, + "Could not perform insert: less than {} free bytes in disk space ({}). " + "Configure this limit with user settings {} or {}", + needed_free_bytes, + free_disk_bytes, + "min_free_disk_bytes_to_throw_insert", + "min_free_disk_ratio_to_throw_insert"); + } + } + auto new_data_part = data.getDataPartBuilder(part_name, data_part_volume, part_dir) .withPartFormat(data.choosePartFormat(expected_size, block.rows())) .withPartInfo(new_part_info) @@ -564,8 +598,6 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( if (data.storage_settings.get()->assign_part_uuids) new_data_part->uuid = UUIDHelpers::generateV4(); - const auto & data_settings = data.getSettings(); - SerializationInfo::Settings settings{data_settings->ratio_of_defaults_for_sparse_serialization, true}; SerializationInfoByName infos(columns, settings); infos.add(block); diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index dabb6991b0b..5d7e1d5107a 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -225,6 +225,14 @@ void MergeTreeSettings::sanityCheck(size_t background_pool_tasks) const "The value of merge_selecting_sleep_slowdown_factor setting ({}) cannot be less than 1.0", merge_selecting_sleep_slowdown_factor); } + + if (min_free_disk_ratio_to_throw_insert < 0.0 || min_free_disk_ratio_to_throw_insert > 1.0) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "The value of setting min_free_disk_ratio_to_throw_insert ({}) must be between 0.0 and 1.0", + min_free_disk_ratio_to_throw_insert); + } } void MergeTreeColumnSettings::validate(const SettingsChanges & changes) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index dcb18155114..29f4c22f196 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -101,6 +101,8 @@ struct Settings; M(Bool, optimize_row_order, false, "Allow reshuffling of rows during part inserts and merges to improve the compressibility of the new part", 0) \ M(Bool, use_adaptive_write_buffer_for_dynamic_subcolumns, true, "Allow to use adaptive writer buffers during writing dynamic subcolumns to reduce memory usage", 0) \ M(UInt64, adaptive_write_buffer_initial_size, 16 * 1024, "Initial size of an adaptive write buffer", 0) \ + M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ + M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ \ /* Part removal settings. */ \ M(UInt64, simultaneous_parts_removal_limit, 0, "Maximum number of parts to remove during one CleanupThread iteration (0 means unlimited).", 0) \ diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml b/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml new file mode 100644 index 00000000000..d4031ff656c --- /dev/null +++ b/tests/integration/test_stop_insert_when_disk_close_to_full/configs/config.d/storage_configuration.xml @@ -0,0 +1,19 @@ + + + + + local + /disk1/ + + + + + +
+ disk1 +
+
+
+
+
+
diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py new file mode 100644 index 00000000000..2ecb7f86fe1 --- /dev/null +++ b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py @@ -0,0 +1,61 @@ +import pytest +from helpers.cluster import ClickHouseCluster, ClickHouseInstance +from helpers.client import QueryRuntimeException + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", + main_configs=["configs/config.d/storage_configuration.xml"], + tmpfs=["/disk1:size=7M"], + macros={"shard": 0, "replica": 1}, +) + + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +def test_insert_stops_when_disk_full(start_cluster): + min_free_bytes = 3 * 1024 * 1024 # 3 MiB + + node.query( + f""" + CREATE TABLE test_table ( + id UInt32, + data String + ) ENGINE = MergeTree() + ORDER BY id + SETTINGS storage_policy = 'only_disk1', min_free_disk_bytes_to_throw_insert = {min_free_bytes} + """ + ) + + count = 0 + + # Insert data to fill up disk + try: + for _ in range(100000): + node.query( + "INSERT INTO test_table SELECT number, repeat('a', 1000 * 1000) FROM numbers(1)" + ) + count += 1 + except QueryRuntimeException as e: + assert "Could not perform insert" in str(e) + assert "free bytes in disk space" in str(e) + + free_space = int( + node.query("SELECT free_space FROM system.disks WHERE name = 'disk1'").strip() + ) + assert ( + free_space <= min_free_bytes + ), f"Free space ({free_space}) is less than min_free_bytes ({min_free_bytes})" + + rows = int(node.query("SELECT count() from test_table").strip()) + assert rows == count + + node.query("DROP TABLE test_table") \ No newline at end of file From d0677e2343920bde3fbac681326209fb25a1e3e6 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Thu, 19 Sep 2024 08:53:47 +0000 Subject: [PATCH 03/65] Added more extensive testing to new settings. --- src/Core/Defines.h | 2 +- .../__init__.py | 0 .../test.py | 62 ++++++++++++++++++- 3 files changed, 62 insertions(+), 2 deletions(-) create mode 100644 tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 81195af6b31..648da23730d 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -108,7 +108,7 @@ static constexpr auto DEFAULT_QUERY_CACHE_MAX_SIZE = 1_GiB; static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRIES = 1024uz; static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_BYTES = 1_MiB; static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS = 30'000'000uz; -static constexpr Float64 DEFAULT_MIN_FREE_DISK_RATIO = 0.0; +static constexpr auto DEFAULT_MIN_FREE_DISK_RATIO = 0.0; /// Query profiler cannot work with sanitizers. /// Sanitizers are using quick "frame walking" stack unwinding (this implies -fno-omit-frame-pointer) diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py b/tests/integration/test_stop_insert_when_disk_close_to_full/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py index 2ecb7f86fe1..8ad9934464f 100644 --- a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py +++ b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py @@ -21,7 +21,67 @@ def start_cluster(): cluster.shutdown() +def test_min_free_disk_settings(start_cluster): + # min_free_disk_bytes_to_throw_insert (default 0) + # min_free_disk_ratio_to_throw_insert (default 0.0) + + node.query("DROP TABLE IF EXISTS test_table") + + node.query( + f""" + CREATE TABLE test_table ( + id UInt32, + data String + ) ENGINE = MergeTree() + ORDER BY id + SETTINGS storage_policy = 'only_disk1' + """ + ) + + node.query("INSERT INTO test_table (id, data) values (1, 'a')") + + free_bytes = 7 * 1024 * 1024 # 7MB -- size of disk + node.query(f"SET min_free_disk_bytes_to_throw_insert = {free_bytes}") + + try: + node.query("INSERT INTO test_table (id, data) values (1, 'a')") + except QueryRuntimeException as e: + assert "NOT_ENOUGH_SPACE" in str(e) + + node.query("SET min_free_disk_bytes_to_throw_insert = 0") + node.query("INSERT INTO test_table (id, data) values (1, 'a')") + + free_ratio = 1.0 + node.query(f"SET min_free_disk_ratio_to_throw_insert = {free_ratio}") + + try: + node.query("INSERT INTO test_table (id, data) values (1, 'a')") + except QueryRuntimeException as e: + assert "NOT_ENOUGH_SPACE" in str(e) + + node.query("DROP TABLE test_table") + + # server setting for min_free_disk_ratio_to_throw_insert is 1 but we can overwrite at table level + node.query( + f""" + CREATE TABLE test_table ( + id UInt32, + data String + ) ENGINE = MergeTree() + ORDER BY id + SETTINGS storage_policy = 'only_disk1', min_free_disk_ratio_to_throw_insert = 0.0 + """ + ) + + node.query("INSERT INTO test_table (id, data) values (1, 'a')") + + node.query("DROP TABLE test_table") + node.query("SET min_free_disk_ratio_to_throw_insert = 0.0") + + def test_insert_stops_when_disk_full(start_cluster): + node.query("DROP TABLE IF EXISTS test_table") + min_free_bytes = 3 * 1024 * 1024 # 3 MiB node.query( @@ -58,4 +118,4 @@ def test_insert_stops_when_disk_full(start_cluster): rows = int(node.query("SELECT count() from test_table").strip()) assert rows == count - node.query("DROP TABLE test_table") \ No newline at end of file + node.query("DROP TABLE test_table") From 7521fd2dcb2050d9837e19b6374dfa0a11f5106e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 19 Sep 2024 18:17:04 +0200 Subject: [PATCH 04/65] Remove unused buggy code --- src/AggregateFunctions/UniquesHashSet.h | 45 ------------------- src/Common/CombinedCardinalityEstimator.h | 42 ----------------- src/Common/HashTable/FixedHashSet.h | 7 --- src/Common/HashTable/HashSet.h | 20 --------- src/Common/HyperLogLogCounter.h | 12 ----- .../HyperLogLogWithSmallSetOptimization.h | 18 -------- 6 files changed, 144 deletions(-) diff --git a/src/AggregateFunctions/UniquesHashSet.h b/src/AggregateFunctions/UniquesHashSet.h index d5241547711..8c05ab36265 100644 --- a/src/AggregateFunctions/UniquesHashSet.h +++ b/src/AggregateFunctions/UniquesHashSet.h @@ -447,51 +447,6 @@ public: } } - void readAndMerge(DB::ReadBuffer & rb) - { - UInt8 rhs_skip_degree = 0; - DB::readBinaryLittleEndian(rhs_skip_degree, rb); - - if (rhs_skip_degree > skip_degree) - { - skip_degree = rhs_skip_degree; - rehash(); - } - - size_t rhs_size = 0; - DB::readVarUInt(rhs_size, rb); - - if (rhs_size > UNIQUES_HASH_MAX_SIZE) - throw Poco::Exception("Cannot read UniquesHashSet: too large size_degree."); - - if ((1ULL << size_degree) < rhs_size) - { - UInt8 new_size_degree = std::max(UNIQUES_HASH_SET_INITIAL_SIZE_DEGREE, static_cast(log2(rhs_size - 1)) + 2); - resize(new_size_degree); - } - - if (rhs_size <= 1) - { - for (size_t i = 0; i < rhs_size; ++i) - { - HashValue x = 0; - DB::readBinaryLittleEndian(x, rb); - insertHash(x); - } - } - else - { - auto hs = std::make_unique(rhs_size); - rb.readStrict(reinterpret_cast(hs.get()), rhs_size * sizeof(HashValue)); - - for (size_t i = 0; i < rhs_size; ++i) - { - DB::transformEndianness(hs[i]); - insertHash(hs[i]); - } - } - } - static void skip(DB::ReadBuffer & rb) { size_t size = 0; diff --git a/src/Common/CombinedCardinalityEstimator.h b/src/Common/CombinedCardinalityEstimator.h index 132f00de8eb..2c3e1e07745 100644 --- a/src/Common/CombinedCardinalityEstimator.h +++ b/src/Common/CombinedCardinalityEstimator.h @@ -177,48 +177,6 @@ public: } } - void readAndMerge(DB::ReadBuffer & in) - { - auto container_type = getContainerType(); - - /// If readAndMerge is called with an empty state, just deserialize - /// the state is specified as a parameter. - if ((container_type == details::ContainerType::SMALL) && small.empty()) - { - read(in); - return; - } - - UInt8 v; - readBinary(v, in); - auto rhs_container_type = static_cast(v); - - auto max_container_type = details::max(container_type, rhs_container_type); - - if (container_type != max_container_type) - { - if (max_container_type == details::ContainerType::MEDIUM) - toMedium(); - else if (max_container_type == details::ContainerType::LARGE) - toLarge(); - } - - if (rhs_container_type == details::ContainerType::SMALL) - { - typename Small::Reader reader(in); - while (reader.next()) - insert(reader.get()); - } - else if (rhs_container_type == details::ContainerType::MEDIUM) - { - typename Medium::Reader reader(in); - while (reader.next()) - insert(reader.get()); - } - else if (rhs_container_type == details::ContainerType::LARGE) - getContainer().readAndMerge(in); - } - void write(DB::WriteBuffer & out) const { auto container_type = getContainerType(); diff --git a/src/Common/HashTable/FixedHashSet.h b/src/Common/HashTable/FixedHashSet.h index e764038e6c3..5b314b1f0a6 100644 --- a/src/Common/HashTable/FixedHashSet.h +++ b/src/Common/HashTable/FixedHashSet.h @@ -16,11 +16,4 @@ public: if (Base::buf[i].isZero(*this) && !rhs.buf[i].isZero(*this)) new (&Base::buf[i]) Cell(rhs.buf[i]); } - - /// NOTE: Currently this method isn't used. When it does, the ReadBuffer should - /// contain the Key explicitly. - // void readAndMerge(DB::ReadBuffer & rb) - // { - - // } }; diff --git a/src/Common/HashTable/HashSet.h b/src/Common/HashTable/HashSet.h index c25bfb14d9c..9b7445d20c6 100644 --- a/src/Common/HashTable/HashSet.h +++ b/src/Common/HashTable/HashSet.h @@ -55,26 +55,6 @@ public: if (!rhs.buf[i].isZero(*this)) this->insert(rhs.buf[i].getValue()); } - - - void readAndMerge(DB::ReadBuffer & rb) - { - Cell::State::read(rb); - - size_t new_size = 0; - DB::readVarUInt(new_size, rb); - if (new_size > 100'000'000'000) - throw DB::Exception(DB::ErrorCodes::TOO_LARGE_ARRAY_SIZE, "The size of serialized hash table is suspiciously large: {}", new_size); - - this->resize(new_size); - - for (size_t i = 0; i < new_size; ++i) - { - Cell x; - x.read(rb); - this->insert(x.getValue()); - } - } }; diff --git a/src/Common/HyperLogLogCounter.h b/src/Common/HyperLogLogCounter.h index 9b2b33dc918..b3e509c782d 100644 --- a/src/Common/HyperLogLogCounter.h +++ b/src/Common/HyperLogLogCounter.h @@ -353,18 +353,6 @@ public: } } - void readAndMerge(DB::ReadBuffer & in) - { - typename RankStore::Reader reader(in); - while (reader.next()) - { - const auto & data = reader.get(); - update(data.first, data.second); - } - - in.ignore(sizeof(DenominatorCalculatorType) + sizeof(ZerosCounterType)); - } - static void skip(DB::ReadBuffer & in) { in.ignore(sizeof(RankStore) + sizeof(DenominatorCalculatorType) + sizeof(ZerosCounterType)); diff --git a/src/Common/HyperLogLogWithSmallSetOptimization.h b/src/Common/HyperLogLogWithSmallSetOptimization.h index 1d2408186de..1748f32cd95 100644 --- a/src/Common/HyperLogLogWithSmallSetOptimization.h +++ b/src/Common/HyperLogLogWithSmallSetOptimization.h @@ -113,24 +113,6 @@ public: small.read(in); } - void readAndMerge(DB::ReadBuffer & in) - { - bool is_rhs_large; - readBinary(is_rhs_large, in); - - if (!isLarge() && is_rhs_large) - toLarge(); - - if (!is_rhs_large) - { - typename Small::Reader reader(in); - while (reader.next()) - insert(reader.get()); - } - else - large->readAndMerge(in); - } - void write(DB::WriteBuffer & out) const { writeBinary(isLarge(), out); From 21261525fa4b664e2ddd7c210d50c9ce38faee86 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Thu, 19 Sep 2024 18:30:39 +0200 Subject: [PATCH 05/65] Update HashSet.h --- src/Common/HashTable/HashSet.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Common/HashTable/HashSet.h b/src/Common/HashTable/HashSet.h index 9b7445d20c6..74c641cd913 100644 --- a/src/Common/HashTable/HashSet.h +++ b/src/Common/HashTable/HashSet.h @@ -16,7 +16,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int TOO_LARGE_ARRAY_SIZE; } } From bd829e2ed008c9d18873d40f1b96ab7326bcfcac Mon Sep 17 00:00:00 2001 From: marco-vb Date: Thu, 19 Sep 2024 18:00:32 +0000 Subject: [PATCH 06/65] Added settings to core --- src/Core/Settings.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 4518d78657c..d7773bb5f09 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -360,6 +360,8 @@ namespace ErrorCodes M(Int64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite. Zero means async mode.", 0) \ M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ + M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ + M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ \ M(Bool, final, false, "Query with the FINAL modifier by default. If the engine does not support final, it does not have any effect. On queries with multiple tables final is applied only on those that support it. It also works on distributed tables", 0) \ \ From ad4adc7c3fbd0b8f120201cdfc50f435e7415b1c Mon Sep 17 00:00:00 2001 From: marco-vb Date: Thu, 19 Sep 2024 19:55:20 +0000 Subject: [PATCH 07/65] Refer to settings the new way. --- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 4134b0999c1..66e4a763946 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -60,6 +60,8 @@ namespace Setting extern const SettingsBool materialize_statistics_on_insert; extern const SettingsBool optimize_on_insert; extern const SettingsBool throw_on_max_partitions_per_insert_block; + extern const SettingsUInt64 min_free_disk_bytes_to_throw_insert; + extern const SettingsDouble min_free_disk_ratio_to_throw_insert; } namespace ErrorCodes @@ -564,11 +566,11 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( const auto & global_settings = context->getSettingsRef(); const auto & data_settings = data.getSettings(); - UInt64 min_bytes = global_settings.min_free_disk_bytes_to_throw_insert; + UInt64 min_bytes = global_settings[Setting::min_free_disk_bytes_to_throw_insert]; if (data_settings->min_free_disk_bytes_to_throw_insert.changed) min_bytes = data_settings->min_free_disk_bytes_to_throw_insert; - Float64 min_ratio = global_settings.min_free_disk_ratio_to_throw_insert; + Float64 min_ratio = global_settings[Setting::min_free_disk_ratio_to_throw_insert]; if (data_settings->min_free_disk_ratio_to_throw_insert.changed) min_ratio = data_settings->min_free_disk_ratio_to_throw_insert; From 84151122a09cba21d40125f31f4def6897089340 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 23 Sep 2024 03:13:00 +0000 Subject: [PATCH 08/65] Fix compilation after merging and update document --- docs/en/operations/settings/settings.md | 6 ++++++ src/Core/Settings.cpp | 2 +- src/Interpreters/executeQuery.cpp | 3 ++- src/Parsers/IAST.cpp | 12 +++++++----- 4 files changed, 16 insertions(+), 7 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e826bb457c2..e572ba65cf4 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -5682,3 +5682,9 @@ Default value: `0`. Enable `IF NOT EXISTS` for `CREATE` statement by default. If either this setting or `IF NOT EXISTS` is specified and a table with the provided name already exists, no exception will be thrown. Default value: `false`. + +## enable_secure_identifiers + +If enabled, only allow secure identifiers which contain only underscore and alphanumeric characters + +Default value: `false`. diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index ad43d66150b..7e919848a0e 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -918,7 +918,7 @@ namespace ErrorCodes M(Bool, restore_replace_external_engines_to_null, false, "Replace all the external table engines to Null on restore. Useful for testing purposes", 0) \ M(Bool, restore_replace_external_table_functions_to_null, false, "Replace all table functions to Null on restore. Useful for testing purposes", 0) \ M(Bool, create_if_not_exists, false, "Enable IF NOT EXISTS for CREATE statements by default", 0) \ - M(Bool, enable_secure_identifiers, false, "If enabled, only secure identifiers are allowed", 0) \ + M(Bool, enable_secure_identifiers, false, "If enabled, only allow secure identifiers which contain only underscore and alphanumeric characters", 0) \ \ \ /* ###################################### */ \ diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 0a45d21eb73..8648608b1a8 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -155,6 +155,7 @@ namespace Setting extern const SettingsBool use_query_cache; extern const SettingsBool wait_for_async_insert; extern const SettingsSeconds wait_for_async_insert_timeout; + extern const SettingsBool enable_secure_identifiers; } namespace ErrorCodes @@ -992,7 +993,7 @@ static std::tuple executeQueryImpl( InterpreterSetQuery::applySettingsFromQuery(ast, context); validateAnalyzerSettings(ast, settings[Setting::allow_experimental_analyzer]); - if (settings.enable_secure_identifiers) + if (settings[Setting::enable_secure_identifiers]) { WriteBufferFromOwnString buf; IAST::FormatSettings enable_secure_identifiers_settings(buf, true); diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 0ea64ccfdbd..b5f3c9b4603 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -6,6 +6,7 @@ #include #include +#include namespace DB { @@ -291,12 +292,13 @@ void IAST::FormatSettings::checkIdentifier(const String & name) const { if (enable_secure_identifiers) { - for (char c : name) + bool is_secure_identifier = std::all_of(name.begin(), name.end(), [](char ch) { return std::isalnum(ch) || ch == '_'; }); + if (!is_secure_identifier) { - if (!std::isalnum(c) && c != '_') - { - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Not a secure identifier: `{}`", name); - } + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Not a secure identifier: `{}`, a secure identifier must contain only underscore and alphanumeric characters", + name); } } } From d91a419e7ce34b44e8c1413726d5996252e5a85f Mon Sep 17 00:00:00 2001 From: marco-vb Date: Mon, 23 Sep 2024 13:34:09 +0000 Subject: [PATCH 09/65] Addressing requested changes on review. --- .../settings/merge-tree-settings.md | 6 ++-- src/Core/Defines.h | 1 - src/Core/Settings.cpp | 4 +-- src/Core/SettingsChangesHistory.cpp | 4 +-- src/Core/SettingsQuirks.cpp | 8 ----- .../MergeTree/MergeTreeDataWriter.cpp | 36 ++++++++++--------- src/Storages/MergeTree/MergeTreeSettings.cpp | 8 ----- src/Storages/MergeTree/MergeTreeSettings.h | 4 +-- .../test.py | 20 +++++------ 9 files changed, 39 insertions(+), 52 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index af8e16c30ea..d2df5123af0 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -1066,11 +1066,12 @@ Possible values: Default value: throw -## min_free_disk_bytes_to_throw_insert +## min_free_disk_bytes_to_perform_insert The minimum number of bytes that should be free in disk space in order to insert data. If the number of available free bytes is less than `min_free_disk_bytes_to_throw_insert` then an exception is thrown and the insert is not executed. Note that this setting: - takes into account the `keep_free_space_bytes` setting. - does not take into account the amount of data that will be written by the `INSERT` operation. +- is only checked if a positive (non-zero) number of bytes is specified Possible values: @@ -1078,11 +1079,12 @@ Possible values: Default value: 0 bytes. -## min_free_disk_ratio_to_throw_insert +## min_free_disk_ratio_to_perform_insert The minimum free to total disk space ratio to perform an `INSERT`. Must be a floating point value between 0 and 1. Note that this setting: - takes into account the `keep_free_space_bytes` setting. - does not take into account the amount of data that will be written by the `INSERT` operation. +- is only checked if a positive (non-zero) ratio is specified Possible values: diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 648da23730d..c6e65f34e90 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -108,7 +108,6 @@ static constexpr auto DEFAULT_QUERY_CACHE_MAX_SIZE = 1_GiB; static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRIES = 1024uz; static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_BYTES = 1_MiB; static constexpr auto DEFAULT_QUERY_CACHE_MAX_ENTRY_SIZE_IN_ROWS = 30'000'000uz; -static constexpr auto DEFAULT_MIN_FREE_DISK_RATIO = 0.0; /// Query profiler cannot work with sanitizers. /// Sanitizers are using quick "frame walking" stack unwinding (this implies -fno-omit-frame-pointer) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 302199cea20..a3a4f415adf 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -360,8 +360,8 @@ namespace ErrorCodes M(Int64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in the cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. A negative value means infinite. Zero means async mode.", 0) \ M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ - M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ - M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ + M(UInt64, min_free_disk_bytes_to_perform_insert, 0, "Minimum free disk space bytes to perform an insert.", 0) \ + M(Double, min_free_disk_ratio_to_perform_insert, 0.0, "Minimum free disk space ratio to perform an insert.", 0) \ \ M(Bool, final, false, "Query with the FINAL modifier by default. If the engine does not support the FINAL, it does not have any effect. On queries with multiple tables, FINAL is applied only to those that support it. It also works on distributed tables", 0) \ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index da0fbfad255..52b2bfa6948 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -86,8 +86,8 @@ static std::initializer_list(); - min_free_disk_ratio_to_throw_insert < 0.0 || min_free_disk_ratio_to_throw_insert > 1) - { - if (log) - LOG_WARNING(log, "Sanity check: 'min_free_disk_ratio_to_throw_insert' must be between 0.0 and 1.0. Set to default (0.0)"); - current_settings.set("min_free_disk_ratio_to_throw_insert", DEFAULT_MIN_FREE_DISK_RATIO); - } } } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 66e4a763946..7592ae624f3 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -60,8 +60,8 @@ namespace Setting extern const SettingsBool materialize_statistics_on_insert; extern const SettingsBool optimize_on_insert; extern const SettingsBool throw_on_max_partitions_per_insert_block; - extern const SettingsUInt64 min_free_disk_bytes_to_throw_insert; - extern const SettingsDouble min_free_disk_ratio_to_throw_insert; + extern const SettingsUInt64 min_free_disk_bytes_to_perform_insert; + extern const SettingsDouble min_free_disk_ratio_to_perform_insert; } namespace ErrorCodes @@ -566,33 +566,35 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( const auto & global_settings = context->getSettingsRef(); const auto & data_settings = data.getSettings(); - UInt64 min_bytes = global_settings[Setting::min_free_disk_bytes_to_throw_insert]; - if (data_settings->min_free_disk_bytes_to_throw_insert.changed) - min_bytes = data_settings->min_free_disk_bytes_to_throw_insert; + const UInt64 & min_bytes_to_perform_insert = + data_settings->min_free_disk_bytes_to_perform_insert.changed + ? data_settings->min_free_disk_bytes_to_perform_insert + : global_settings[Setting::min_free_disk_bytes_to_perform_insert]; - Float64 min_ratio = global_settings[Setting::min_free_disk_ratio_to_throw_insert]; - if (data_settings->min_free_disk_ratio_to_throw_insert.changed) - min_ratio = data_settings->min_free_disk_ratio_to_throw_insert; + const Float64 & min_ratio_to_perform_insert = + data_settings->min_free_disk_ratio_to_perform_insert.changed + ? data_settings->min_free_disk_ratio_to_perform_insert + : global_settings[Setting::min_free_disk_ratio_to_perform_insert]; - if (min_bytes > 0 || min_ratio > 0.0) + if (min_bytes_to_perform_insert > 0 || min_ratio_to_perform_insert > 0.0) { - const auto disk = data_part_volume->getDisk(); - const UInt64 total_disk_bytes = *disk->getTotalSpace(); - const UInt64 free_disk_bytes = *disk->getAvailableSpace(); + const auto & disk = data_part_volume->getDisk(); + const UInt64 & total_disk_bytes = disk->getTotalSpace().value_or(0); + const UInt64 & free_disk_bytes = disk->getAvailableSpace().value_or(0.0); - const UInt64 min_bytes_from_ratio = static_cast(min_ratio * total_disk_bytes); - const UInt64 needed_free_bytes = std::max(min_bytes, min_bytes_from_ratio); + const UInt64 & min_bytes_from_ratio = static_cast(min_ratio_to_perform_insert * total_disk_bytes); + const UInt64 & needed_free_bytes = std::max(min_bytes_to_perform_insert, min_bytes_from_ratio); if (needed_free_bytes > free_disk_bytes) { throw Exception( ErrorCodes::NOT_ENOUGH_SPACE, - "Could not perform insert: less than {} free bytes in disk space ({}). " + "Could not perform insert: less than {} free bytes left in the disk space ({}). " "Configure this limit with user settings {} or {}", needed_free_bytes, free_disk_bytes, - "min_free_disk_bytes_to_throw_insert", - "min_free_disk_ratio_to_throw_insert"); + "min_free_disk_bytes_to_perform_insert", + "min_free_disk_ratio_to_perform_insert"); } } diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 1a23ba7bb14..6beb0927cbf 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -225,14 +225,6 @@ void MergeTreeSettings::sanityCheck(size_t background_pool_tasks) const "The value of merge_selecting_sleep_slowdown_factor setting ({}) cannot be less than 1.0", merge_selecting_sleep_slowdown_factor); } - - if (min_free_disk_ratio_to_throw_insert < 0.0 || min_free_disk_ratio_to_throw_insert > 1.0) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "The value of setting min_free_disk_ratio_to_throw_insert ({}) must be between 0.0 and 1.0", - min_free_disk_ratio_to_throw_insert); - } } void MergeTreeColumnSettings::validate(const SettingsChanges & changes) diff --git a/src/Storages/MergeTree/MergeTreeSettings.h b/src/Storages/MergeTree/MergeTreeSettings.h index a22bd9cab2a..add20b7cf75 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.h +++ b/src/Storages/MergeTree/MergeTreeSettings.h @@ -101,8 +101,8 @@ struct Settings; M(Bool, optimize_row_order, false, "Allow reshuffling of rows during part inserts and merges to improve the compressibility of the new part", 0) \ M(Bool, use_adaptive_write_buffer_for_dynamic_subcolumns, true, "Allow to use adaptive writer buffers during writing dynamic subcolumns to reduce memory usage", 0) \ M(UInt64, adaptive_write_buffer_initial_size, 16 * 1024, "Initial size of an adaptive write buffer", 0) \ - M(UInt64, min_free_disk_bytes_to_throw_insert, 0, "Minimum free disk space bytes to throw an insert.", 0) \ - M(Double, min_free_disk_ratio_to_throw_insert, 0.0, "Minimum free disk space ratio to throw an insert.", 0) \ + M(UInt64, min_free_disk_bytes_to_perform_insert, 0, "Minimum free disk space bytes to perform an insert.", 0) \ + M(Double, min_free_disk_ratio_to_perform_insert, 0.0, "Minimum free disk space ratio to perform an insert.", 0) \ \ /* Part removal settings. */ \ M(UInt64, simultaneous_parts_removal_limit, 0, "Maximum number of parts to remove during one CleanupThread iteration (0 means unlimited).", 0) \ diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py index 8ad9934464f..82c36adda51 100644 --- a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py +++ b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py @@ -22,8 +22,8 @@ def start_cluster(): def test_min_free_disk_settings(start_cluster): - # min_free_disk_bytes_to_throw_insert (default 0) - # min_free_disk_ratio_to_throw_insert (default 0.0) + # min_free_disk_bytes_to_perform_insert (default 0) + # min_free_disk_ratio_to_perform_insert (default 0.0) node.query("DROP TABLE IF EXISTS test_table") @@ -41,18 +41,18 @@ def test_min_free_disk_settings(start_cluster): node.query("INSERT INTO test_table (id, data) values (1, 'a')") free_bytes = 7 * 1024 * 1024 # 7MB -- size of disk - node.query(f"SET min_free_disk_bytes_to_throw_insert = {free_bytes}") + node.query(f"SET min_free_disk_bytes_to_perform_insert = {free_bytes}") try: node.query("INSERT INTO test_table (id, data) values (1, 'a')") except QueryRuntimeException as e: assert "NOT_ENOUGH_SPACE" in str(e) - node.query("SET min_free_disk_bytes_to_throw_insert = 0") + node.query("SET min_free_disk_bytes_to_perform_insert = 0") node.query("INSERT INTO test_table (id, data) values (1, 'a')") free_ratio = 1.0 - node.query(f"SET min_free_disk_ratio_to_throw_insert = {free_ratio}") + node.query(f"SET min_free_disk_ratio_to_perform_insert = {free_ratio}") try: node.query("INSERT INTO test_table (id, data) values (1, 'a')") @@ -61,7 +61,7 @@ def test_min_free_disk_settings(start_cluster): node.query("DROP TABLE test_table") - # server setting for min_free_disk_ratio_to_throw_insert is 1 but we can overwrite at table level + # server setting for min_free_disk_ratio_to_perform_insert is 1 but we can overwrite at table level node.query( f""" CREATE TABLE test_table ( @@ -69,14 +69,14 @@ def test_min_free_disk_settings(start_cluster): data String ) ENGINE = MergeTree() ORDER BY id - SETTINGS storage_policy = 'only_disk1', min_free_disk_ratio_to_throw_insert = 0.0 + SETTINGS storage_policy = 'only_disk1', min_free_disk_ratio_to_perform_insert = 0.0 """ ) node.query("INSERT INTO test_table (id, data) values (1, 'a')") node.query("DROP TABLE test_table") - node.query("SET min_free_disk_ratio_to_throw_insert = 0.0") + node.query("SET min_free_disk_ratio_to_perform_insert = 0.0") def test_insert_stops_when_disk_full(start_cluster): @@ -91,7 +91,7 @@ def test_insert_stops_when_disk_full(start_cluster): data String ) ENGINE = MergeTree() ORDER BY id - SETTINGS storage_policy = 'only_disk1', min_free_disk_bytes_to_throw_insert = {min_free_bytes} + SETTINGS storage_policy = 'only_disk1', min_free_disk_bytes_to_perform_insert = {min_free_bytes} """ ) @@ -106,7 +106,7 @@ def test_insert_stops_when_disk_full(start_cluster): count += 1 except QueryRuntimeException as e: assert "Could not perform insert" in str(e) - assert "free bytes in disk space" in str(e) + assert "free bytes left in the disk space" in str(e) free_space = int( node.query("SELECT free_space FROM system.disks WHERE name = 'disk1'").strip() From ee1f3904a13e232cff43dcd188b665123edd969c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 23 Sep 2024 17:04:26 +0000 Subject: [PATCH 10/65] better performance in parsing with missing values --- src/Core/BlockInfo.cpp | 32 ----------- src/Core/BlockInfo.h | 30 ----------- src/Core/BlockMissingValues.cpp | 53 +++++++++++++++++++ src/Core/BlockMissingValues.h | 44 +++++++++++++++ src/Formats/NativeReader.h | 1 + src/Formats/insertNullAsDefaultIfNeeded.h | 1 + src/Processors/Chunk.cpp | 16 ------ src/Processors/Chunk.h | 22 -------- src/Processors/Formats/IInputFormat.h | 7 +-- src/Processors/Formats/IRowInputFormat.cpp | 5 +- src/Processors/Formats/IRowInputFormat.h | 2 +- .../Formats/Impl/ArrowBlockInputFormat.cpp | 9 ++-- .../Formats/Impl/ArrowBlockInputFormat.h | 2 +- .../Formats/Impl/ArrowColumnToCHColumn.h | 1 + .../Impl/JSONColumnsBlockInputFormatBase.cpp | 1 + .../Impl/JSONColumnsBlockInputFormatBase.h | 2 +- src/Processors/Formats/Impl/NativeFormat.cpp | 7 ++- .../Impl/NativeORCBlockInputFormat.cpp | 9 ++-- .../Formats/Impl/NativeORCBlockInputFormat.h | 2 +- .../Formats/Impl/ORCBlockInputFormat.cpp | 9 ++-- .../Formats/Impl/ORCBlockInputFormat.h | 2 +- .../Impl/ParallelParsingInputFormat.cpp | 7 ++- .../Formats/Impl/ParallelParsingInputFormat.h | 7 +-- .../Formats/Impl/ParquetBlockInputFormat.cpp | 31 +++++------ .../Formats/Impl/ParquetBlockInputFormat.h | 4 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 1 + .../Formats/Impl/ValuesBlockInputFormat.h | 2 +- .../Transforms/AddingDefaultsTransform.cpp | 8 +-- tests/performance/insert_sparse_column.xml | 13 +++++ 29 files changed, 181 insertions(+), 149 deletions(-) create mode 100644 src/Core/BlockMissingValues.cpp create mode 100644 src/Core/BlockMissingValues.h create mode 100644 tests/performance/insert_sparse_column.xml diff --git a/src/Core/BlockInfo.cpp b/src/Core/BlockInfo.cpp index a5e08771c4e..9de229328fe 100644 --- a/src/Core/BlockInfo.cpp +++ b/src/Core/BlockInfo.cpp @@ -58,36 +58,4 @@ void BlockInfo::read(ReadBuffer & in) } } -void BlockMissingValues::setBit(size_t column_idx, size_t row_idx) -{ - RowsBitMask & mask = rows_mask_by_column_id[column_idx]; - mask.resize(row_idx + 1); - mask[row_idx] = true; -} - -void BlockMissingValues::setBits(size_t column_idx, size_t rows) -{ - RowsBitMask & mask = rows_mask_by_column_id[column_idx]; - mask.resize(rows); - std::fill(mask.begin(), mask.end(), true); -} - -const BlockMissingValues::RowsBitMask & BlockMissingValues::getDefaultsBitmask(size_t column_idx) const -{ - static RowsBitMask none; - auto it = rows_mask_by_column_id.find(column_idx); - if (it != rows_mask_by_column_id.end()) - return it->second; - return none; -} - -bool BlockMissingValues::hasDefaultBits(size_t column_idx) const -{ - auto it = rows_mask_by_column_id.find(column_idx); - if (it == rows_mask_by_column_id.end()) - return false; - - const auto & col_mask = it->second; - return std::find(col_mask.begin(), col_mask.end(), true) != col_mask.end(); -} } diff --git a/src/Core/BlockInfo.h b/src/Core/BlockInfo.h index d431303ca39..f39a6c12df8 100644 --- a/src/Core/BlockInfo.h +++ b/src/Core/BlockInfo.h @@ -2,10 +2,6 @@ #include -#include -#include - - namespace DB { @@ -46,30 +42,4 @@ struct BlockInfo void read(ReadBuffer & in); }; -/// Block extension to support delayed defaults. AddingDefaultsTransform uses it to replace missing values with column defaults. -class BlockMissingValues -{ -public: - using RowsBitMask = std::vector; /// a bit per row for a column - - /// Get mask for column, column_idx is index inside corresponding block - const RowsBitMask & getDefaultsBitmask(size_t column_idx) const; - /// Check that we have to replace default value at least in one of columns - bool hasDefaultBits(size_t column_idx) const; - /// Set bit for a specified row in a single column. - void setBit(size_t column_idx, size_t row_idx); - /// Set bits for all rows in a single column. - void setBits(size_t column_idx, size_t rows); - bool empty() const { return rows_mask_by_column_id.empty(); } - size_t size() const { return rows_mask_by_column_id.size(); } - void clear() { rows_mask_by_column_id.clear(); } - -private: - using RowsMaskByColumnId = std::unordered_map; - - /// If rows_mask_by_column_id[column_id][row_id] is true related value in Block should be replaced with column default. - /// It could contain less columns and rows then related block. - RowsMaskByColumnId rows_mask_by_column_id; -}; - } diff --git a/src/Core/BlockMissingValues.cpp b/src/Core/BlockMissingValues.cpp new file mode 100644 index 00000000000..c37d512e9ee --- /dev/null +++ b/src/Core/BlockMissingValues.cpp @@ -0,0 +1,53 @@ +#include + +namespace DB +{ + +void BlockMissingValues::setBit(size_t column_idx, size_t row_idx) +{ + RowsBitMask & mask = rows_mask_by_column_id[column_idx]; + mask.resize(row_idx + 1); + mask.set(row_idx, true); +} + +void BlockMissingValues::setBits(size_t column_idx, size_t rows) +{ + auto & mask = rows_mask_by_column_id[column_idx]; + mask.set(0, std::min(mask.size(), rows), true); + mask.resize(rows, true); +} + +const BlockMissingValues::RowsBitMask & BlockMissingValues::getDefaultsBitmask(size_t column_idx) const +{ + return rows_mask_by_column_id[column_idx]; +} + +bool BlockMissingValues::hasDefaultBits(size_t column_idx) const +{ + /// It is correct because we resize bitmask only when set a bit. + return !rows_mask_by_column_id[column_idx].empty(); +} + +void BlockMissingValues::clear() +{ + for (auto & mask : rows_mask_by_column_id) + mask.clear(); +} + +bool BlockMissingValues::empty() const +{ + return std::ranges::all_of(rows_mask_by_column_id, [&](const auto & mask) + { + return mask.empty(); + }); +} + +size_t BlockMissingValues::size() const +{ + size_t res = 0; + for (const auto & mask : rows_mask_by_column_id) + res += !mask.empty(); + return res; +} + +} diff --git a/src/Core/BlockMissingValues.h b/src/Core/BlockMissingValues.h new file mode 100644 index 00000000000..cf5e482a569 --- /dev/null +++ b/src/Core/BlockMissingValues.h @@ -0,0 +1,44 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/// Block extension to support delayed defaults. +/// AddingDefaultsTransform uses it to replace missing values with column defaults. +class BlockMissingValues +{ +public: + using RowsBitMask = boost::dynamic_bitset<>; /// a bit per row for a column + explicit BlockMissingValues(size_t num_columns) : rows_mask_by_column_id(num_columns) {} + + /// Get mask for column, column_idx is index inside corresponding block + const RowsBitMask & getDefaultsBitmask(size_t column_idx) const; + /// Check that we have to replace default value at least in one of columns + bool hasDefaultBits(size_t column_idx) const; + /// Set bit for a specified row in a single column. + void setBit(size_t column_idx, size_t row_idx); + /// Set bits for all rows in a single column. + void setBits(size_t column_idx, size_t rows); + + void clear(); + bool empty() const; + size_t size() const; + +private: + using RowsMaskByColumnId = std::vector; + + /// If rows_mask_by_column_id[column_id][row_id] is true related value in Block should be replaced with column default. + /// It could contain less rows than related block. + RowsMaskByColumnId rows_mask_by_column_id; +}; + +/// The same as above but can be used as a chunk info. +class ChunkMissingValues : public BlockMissingValues, public ChunkInfoCloneable +{ +}; + +} diff --git a/src/Formats/NativeReader.h b/src/Formats/NativeReader.h index 97b6ea22b15..a582ba1b321 100644 --- a/src/Formats/NativeReader.h +++ b/src/Formats/NativeReader.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Formats/insertNullAsDefaultIfNeeded.h b/src/Formats/insertNullAsDefaultIfNeeded.h index 874f803a14c..d8c96c30c3e 100644 --- a/src/Formats/insertNullAsDefaultIfNeeded.h +++ b/src/Formats/insertNullAsDefaultIfNeeded.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB { diff --git a/src/Processors/Chunk.cpp b/src/Processors/Chunk.cpp index 4466be5b3a7..43eb0df87d4 100644 --- a/src/Processors/Chunk.cpp +++ b/src/Processors/Chunk.cpp @@ -178,22 +178,6 @@ void Chunk::append(const Chunk & chunk, size_t from, size_t length) setColumns(std::move(mutable_columns), rows); } -void ChunkMissingValues::setBit(size_t column_idx, size_t row_idx) -{ - RowsBitMask & mask = rows_mask_by_column_id[column_idx]; - mask.resize(row_idx + 1); - mask[row_idx] = true; -} - -const ChunkMissingValues::RowsBitMask & ChunkMissingValues::getDefaultsBitmask(size_t column_idx) const -{ - static RowsBitMask none; - auto it = rows_mask_by_column_id.find(column_idx); - if (it != rows_mask_by_column_id.end()) - return it->second; - return none; -} - void convertToFullIfConst(Chunk & chunk) { size_t num_rows = chunk.getNumRows(); diff --git a/src/Processors/Chunk.h b/src/Processors/Chunk.h index f45e2c4619e..fd3bae45c69 100644 --- a/src/Processors/Chunk.h +++ b/src/Processors/Chunk.h @@ -153,28 +153,6 @@ public: using AsyncInsertInfoPtr = std::shared_ptr; -/// Extension to support delayed defaults. AddingDefaultsProcessor uses it to replace missing values with column defaults. -class ChunkMissingValues : public ChunkInfoCloneable -{ -public: - ChunkMissingValues(const ChunkMissingValues & other) = default; - - using RowsBitMask = std::vector; /// a bit per row for a column - - const RowsBitMask & getDefaultsBitmask(size_t column_idx) const; - void setBit(size_t column_idx, size_t row_idx); - bool empty() const { return rows_mask_by_column_id.empty(); } - size_t size() const { return rows_mask_by_column_id.size(); } - void clear() { rows_mask_by_column_id.clear(); } - -private: - using RowsMaskByColumnId = std::unordered_map; - - /// If rows_mask_by_column_id[column_id][row_id] is true related value in Block should be replaced with column default. - /// It could contain less columns and rows then related block. - RowsMaskByColumnId rows_mask_by_column_id; -}; - /// Converts all columns to full serialization in chunk. /// It's needed, when you have to access to the internals of the column, /// or when you need to perform operation with two columns diff --git a/src/Processors/Formats/IInputFormat.h b/src/Processors/Formats/IInputFormat.h index 713c1089d28..febb0f76d97 100644 --- a/src/Processors/Formats/IInputFormat.h +++ b/src/Processors/Formats/IInputFormat.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -43,11 +44,7 @@ public: virtual void setReadBuffer(ReadBuffer & in_); virtual void resetReadBuffer() { in = nullptr; } - virtual const BlockMissingValues & getMissingValues() const - { - static const BlockMissingValues none; - return none; - } + virtual const BlockMissingValues * getMissingValues() const { return nullptr; } /// Must be called from ParallelParsingInputFormat after readSuffix ColumnMappingPtr getColumnMapping() const { return column_mapping; } diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 0b6c81923db..2d6c6a78e9b 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -56,7 +56,10 @@ bool isParseError(int code) } IRowInputFormat::IRowInputFormat(Block header, ReadBuffer & in_, Params params_) - : IInputFormat(std::move(header), &in_), serializations(getPort().getHeader().getSerializations()), params(params_) + : IInputFormat(std::move(header), &in_) + , serializations(getPort().getHeader().getSerializations()) + , params(params_) + , block_missing_values(getPort().getHeader().columns()) { } diff --git a/src/Processors/Formats/IRowInputFormat.h b/src/Processors/Formats/IRowInputFormat.h index f8796df8604..23dc45d0cff 100644 --- a/src/Processors/Formats/IRowInputFormat.h +++ b/src/Processors/Formats/IRowInputFormat.h @@ -77,7 +77,7 @@ protected: void logError(); - const BlockMissingValues & getMissingValues() const override { return block_missing_values; } + const BlockMissingValues * getMissingValues() const override { return &block_missing_values; } size_t getRowNum() const { return total_rows; } diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index cf079e52db0..1feb6378f9e 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -24,7 +24,10 @@ namespace ErrorCodes } ArrowBlockInputFormat::ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_, bool stream_, const FormatSettings & format_settings_) - : IInputFormat(header_, &in_), stream{stream_}, format_settings(format_settings_) + : IInputFormat(header_, &in_) + , stream(stream_) + , block_missing_values(getPort().getHeader().columns()) + , format_settings(format_settings_) { } @@ -108,9 +111,9 @@ void ArrowBlockInputFormat::resetParser() block_missing_values.clear(); } -const BlockMissingValues & ArrowBlockInputFormat::getMissingValues() const +const BlockMissingValues * ArrowBlockInputFormat::getMissingValues() const { - return block_missing_values; + return &block_missing_values; } static std::shared_ptr createStreamReader(ReadBuffer & in) diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index cb74a9dd93e..295302299dc 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -25,7 +25,7 @@ public: String getName() const override { return "ArrowBlockInputFormat"; } - const BlockMissingValues & getMissingValues() const override; + const BlockMissingValues * getMissingValues() const override; size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; } diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 8521cd2f410..5adc8339ad7 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -9,6 +9,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp index e61e55efc8e..560e6c11c0d 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp +++ b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.cpp @@ -84,6 +84,7 @@ JSONColumnsBlockInputFormatBase::JSONColumnsBlockInputFormatBase( , fields(header_.getNamesAndTypes()) , serializations(header_.getSerializations()) , reader(std::move(reader_)) + , block_missing_values(getPort().getHeader().columns()) { name_to_index = getPort().getHeader().getNamesToIndexesMap(); } diff --git a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h index b1fd86bb806..b9b34bc89ea 100644 --- a/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h +++ b/src/Processors/Formats/Impl/JSONColumnsBlockInputFormatBase.h @@ -52,7 +52,7 @@ public: void setReadBuffer(ReadBuffer & in_) override; - const BlockMissingValues & getMissingValues() const override { return block_missing_values; } + const BlockMissingValues * getMissingValues() const override { return &block_missing_values; } size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; } diff --git a/src/Processors/Formats/Impl/NativeFormat.cpp b/src/Processors/Formats/Impl/NativeFormat.cpp index 38fac60eef6..5411e2e7811 100644 --- a/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/src/Processors/Formats/Impl/NativeFormat.cpp @@ -23,7 +23,10 @@ public: 0, settings, settings.defaults_for_omitted_fields ? &block_missing_values : nullptr)) - , header(header_) {} + , header(header_) + , block_missing_values(header.columns()) + { + } String getName() const override { return "Native"; } @@ -56,7 +59,7 @@ public: IInputFormat::setReadBuffer(in_); } - const BlockMissingValues & getMissingValues() const override { return block_missing_values; } + const BlockMissingValues * getMissingValues() const override { return &block_missing_values; } size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; } diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp index 79dd6e1d35b..adf1b681bae 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.cpp @@ -844,7 +844,10 @@ static void updateIncludeTypeIds( } NativeORCBlockInputFormat::NativeORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_) - : IInputFormat(std::move(header_), &in_), format_settings(format_settings_), skip_stripes(format_settings.orc.skip_stripes) + : IInputFormat(std::move(header_), &in_) + , block_missing_values(getPort().getHeader().columns()) + , format_settings(format_settings_) + , skip_stripes(format_settings.orc.skip_stripes) { } @@ -973,9 +976,9 @@ void NativeORCBlockInputFormat::resetParser() block_missing_values.clear(); } -const BlockMissingValues & NativeORCBlockInputFormat::getMissingValues() const +const BlockMissingValues * NativeORCBlockInputFormat::getMissingValues() const { - return block_missing_values; + return &block_missing_values; } NativeORCSchemaReader::NativeORCSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) diff --git a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h index e4f2ef9ebe3..a1b93b7b995 100644 --- a/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/NativeORCBlockInputFormat.h @@ -57,7 +57,7 @@ public: void resetParser() override; - const BlockMissingValues & getMissingValues() const override; + const BlockMissingValues * getMissingValues() const override; size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; } diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 2266c0b488c..a7b22fe4592 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -23,7 +23,10 @@ namespace ErrorCodes } ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_) - : IInputFormat(std::move(header_), &in_), format_settings(format_settings_), skip_stripes(format_settings.orc.skip_stripes) + : IInputFormat(std::move(header_), &in_) + , block_missing_values(getPort().getHeader().columns()) + , format_settings(format_settings_) + , skip_stripes(format_settings.orc.skip_stripes) { } @@ -86,9 +89,9 @@ void ORCBlockInputFormat::resetParser() block_missing_values.clear(); } -const BlockMissingValues & ORCBlockInputFormat::getMissingValues() const +const BlockMissingValues * ORCBlockInputFormat::getMissingValues() const { - return block_missing_values; + return &block_missing_values; } diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index 85f1636d3dc..63aa5b6eb98 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -27,7 +27,7 @@ public: void resetParser() override; - const BlockMissingValues & getMissingValues() const override; + const BlockMissingValues * getMissingValues() const override; size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; } diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp index 447adb1ed48..bb7efa4913c 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.cpp @@ -110,7 +110,12 @@ void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupPtr thread_grou /// Variable chunk is moved, but it is not really used in the next iteration. /// NOLINTNEXTLINE(bugprone-use-after-move, hicpp-invalid-access-moved) unit.chunk_ext.chunk.emplace_back(std::move(chunk)); - unit.chunk_ext.block_missing_values.emplace_back(parser.getMissingValues()); + + if (const auto * block_missing_values = parser.getMissingValues()) + unit.chunk_ext.block_missing_values.emplace_back(*block_missing_values); + else + unit.chunk_ext.block_missing_values.emplace_back(chunk.getNumColumns()); + size_t approx_chunk_size = input_format->getApproxBytesReadForChunk(); /// We could decompress data during file segmentation. /// Correct chunk size using original segment size. diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h index b97bf5213e6..4c7979a41a0 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h @@ -103,6 +103,7 @@ public: , format_settings(params.format_settings) , min_chunk_bytes(params.min_chunk_bytes) , max_block_size(params.max_block_size) + , last_block_missing_values(getPort().getHeader().columns()) , is_server(params.is_server) , pool(CurrentMetrics::ParallelParsingInputFormatThreads, CurrentMetrics::ParallelParsingInputFormatThreadsActive, CurrentMetrics::ParallelParsingInputFormatThreadsScheduled, params.max_threads) { @@ -124,9 +125,9 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "resetParser() is not allowed for {}", getName()); } - const BlockMissingValues & getMissingValues() const final + const BlockMissingValues * getMissingValues() const final { - return last_block_missing_values; + return &last_block_missing_values; } size_t getApproxBytesReadForChunk() const override { return last_approx_bytes_read_for_chunk; } @@ -190,7 +191,7 @@ private: } } - const BlockMissingValues & getMissingValues() const { return input_format->getMissingValues(); } + const BlockMissingValues * getMissingValues() const { return input_format->getMissingValues(); } private: const InputFormatPtr & input_format; diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 7b82fdf31b4..82d5f186a9e 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -441,6 +441,7 @@ ParquetBlockInputFormat::ParquetBlockInputFormat( , max_decoding_threads(max_decoding_threads_) , min_bytes_for_seek(min_bytes_for_seek_) , pending_chunks(PendingChunk::Compare { .row_group_first = format_settings_.parquet.preserve_order }) + , previous_block_missing_values(getPort().getHeader().columns()) { if (max_decoding_threads > 1) pool = std::make_unique(CurrentMetrics::ParquetDecoderThreads, CurrentMetrics::ParquetDecoderThreadsActive, CurrentMetrics::ParquetDecoderThreadsScheduled, max_decoding_threads); @@ -680,23 +681,19 @@ void ParquetBlockInputFormat::decodeOneChunk(size_t row_group_batch_idx, std::un // reached. Wake up read() instead. condvar.notify_all(); }; - auto get_pending_chunk = [&](size_t num_rows, Chunk chunk = {}) + + auto get_approx_original_chunk_size = [&](size_t num_rows) { - size_t approx_chunk_original_size = static_cast(std::ceil( - static_cast(row_group_batch.total_bytes_compressed) / row_group_batch.total_rows * num_rows)); - return PendingChunk{ - .chunk = std::move(chunk), - .block_missing_values = {}, - .chunk_idx = row_group_batch.next_chunk_idx, - .row_group_batch_idx = row_group_batch_idx, - .approx_original_chunk_size = approx_chunk_original_size - }; + return static_cast(std::ceil(static_cast(row_group_batch.total_bytes_compressed) / row_group_batch.total_rows * num_rows)); }; if (!row_group_batch.record_batch_reader && !row_group_batch.native_record_reader) initializeRowGroupBatchReader(row_group_batch_idx); - PendingChunk res; + PendingChunk res(getPort().getHeader().columns()); + res.chunk_idx = row_group_batch.next_chunk_idx; + res.row_group_batch_idx = row_group_batch_idx; + if (format_settings.parquet.use_native_reader) { auto chunk = row_group_batch.native_record_reader->readChunk(); @@ -706,9 +703,9 @@ void ParquetBlockInputFormat::decodeOneChunk(size_t row_group_batch_idx, std::un return; } - // TODO support defaults_for_omitted_fields feature when supporting nested columns - auto num_rows = chunk.getNumRows(); - res = get_pending_chunk(num_rows, std::move(chunk)); + /// TODO: support defaults_for_omitted_fields feature when supporting nested columns + res.approx_original_chunk_size = get_approx_original_chunk_size(chunk.getNumRows()); + res.chunk = std::move(chunk); } else { @@ -723,11 +720,11 @@ void ParquetBlockInputFormat::decodeOneChunk(size_t row_group_batch_idx, std::un } auto tmp_table = arrow::Table::FromRecordBatches({*batch}); - res = get_pending_chunk((*tmp_table)->num_rows()); /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. /// Otherwise fill the missing columns with zero values of its type. BlockMissingValues * block_missing_values_ptr = format_settings.defaults_for_omitted_fields ? &res.block_missing_values : nullptr; + res.approx_original_chunk_size = get_approx_original_chunk_size((*tmp_table)->num_rows()); res.chunk = row_group_batch.arrow_column_to_ch_column->arrowTableToCHChunk(*tmp_table, (*tmp_table)->num_rows(), block_missing_values_ptr); } @@ -841,9 +838,9 @@ void ParquetBlockInputFormat::resetParser() IInputFormat::resetParser(); } -const BlockMissingValues & ParquetBlockInputFormat::getMissingValues() const +const BlockMissingValues * ParquetBlockInputFormat::getMissingValues() const { - return previous_block_missing_values; + return &previous_block_missing_values; } ParquetSchemaReader::ParquetSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index a83c172c2ed..67439129047 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -63,7 +63,7 @@ public: String getName() const override { return "ParquetBlockInputFormat"; } - const BlockMissingValues & getMissingValues() const override; + const BlockMissingValues * getMissingValues() const override; size_t getApproxBytesReadForChunk() const override { return previous_approx_bytes_read_for_chunk; } @@ -226,6 +226,8 @@ private: // Chunk ready to be delivered by read(). struct PendingChunk { + explicit PendingChunk(size_t num_columns) : block_missing_values(num_columns) {} + Chunk chunk; BlockMissingValues block_missing_values; size_t chunk_idx; // within row group diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 444fe1ce83d..95bc8d4edb7 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -57,6 +57,7 @@ ValuesBlockInputFormat::ValuesBlockInputFormat( parser_type_for_column(num_columns, ParserType::Streaming), attempts_to_deduce_template(num_columns), attempts_to_deduce_template_cached(num_columns), rows_parsed_using_template(num_columns), templates(num_columns), types(header_.getDataTypes()), serializations(header_.getSerializations()) + , block_missing_values(getPort().getHeader().columns()) { } diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index 228e8d0d572..df104d733d5 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -40,7 +40,7 @@ public: void setContext(const ContextPtr & context_) { context = Context::createCopy(context_); } void setQueryParameters(const NameToNameMap & parameters); - const BlockMissingValues & getMissingValues() const override { return block_missing_values; } + const BlockMissingValues * getMissingValues() const override { return &block_missing_values; } size_t getApproxBytesReadForChunk() const override { return approx_bytes_read_for_chunk; } diff --git a/src/Processors/Transforms/AddingDefaultsTransform.cpp b/src/Processors/Transforms/AddingDefaultsTransform.cpp index da4d3a0041b..e7bdf8b6ff3 100644 --- a/src/Processors/Transforms/AddingDefaultsTransform.cpp +++ b/src/Processors/Transforms/AddingDefaultsTransform.cpp @@ -149,8 +149,8 @@ void AddingDefaultsTransform::transform(Chunk & chunk) if (column_defaults.empty()) return; - const BlockMissingValues & block_missing_values = input_format.getMissingValues(); - if (block_missing_values.empty()) + const auto * block_missing_values = input_format.getMissingValues(); + if (!block_missing_values) return; const auto & header = getOutputPort().getHeader(); @@ -167,7 +167,7 @@ void AddingDefaultsTransform::transform(Chunk & chunk) if (evaluate_block.has(column.first)) { size_t column_idx = res.getPositionByName(column.first); - if (block_missing_values.hasDefaultBits(column_idx)) + if (block_missing_values->hasDefaultBits(column_idx)) evaluate_block.erase(column.first); } } @@ -193,7 +193,7 @@ void AddingDefaultsTransform::transform(Chunk & chunk) size_t block_column_position = res.getPositionByName(column_name); ColumnWithTypeAndName & column_read = res.getByPosition(block_column_position); - const auto & defaults_mask = block_missing_values.getDefaultsBitmask(block_column_position); + const auto & defaults_mask = block_missing_values->getDefaultsBitmask(block_column_position); checkCalculated(column_read, column_def, defaults_mask.size()); diff --git a/tests/performance/insert_sparse_column.xml b/tests/performance/insert_sparse_column.xml new file mode 100644 index 00000000000..acc94e418f5 --- /dev/null +++ b/tests/performance/insert_sparse_column.xml @@ -0,0 +1,13 @@ + + CREATE TABLE t_insert_sparse (id UInt64, c0 String, c1 String, c2 String, c3 String, c4 String, c5 String, c6 String, c7 String, c8 String, c9 String, c10 String, c11 String, c12 String, c13 String, c14 String, c15 String, c16 String, c17 String, c18 String, c19 String, c20 String, c21 String, c22 String, c23 String, c24 String, c25 String, c26 String, c27 String, c28 String, c29 String, c30 String, c31 String, c32 String, c33 String, c34 String, c35 String, c36 String, c37 String, c38 String, c39 String, c40 String, c41 String, c42 String, c43 String, c44 String, c45 String, c46 String, c47 String, c48 String, c49 String, c50 String, c51 String, c52 String, c53 String, c54 String, c55 String, c56 String, c57 String, c58 String, c59 String, c60 String, c61 String, c62 String, c63 String, c64 String, c65 String, c66 String, c67 String, c68 String, c69 String, c70 String, c71 String, c72 String, c73 String, c74 String, c75 String, c76 String, c77 String, c78 String, c79 String, c80 String, c81 String, c82 String, c83 String, c84 String, c85 String, c86 String, c87 String, c88 String, c89 String, c90 String, c91 String, c92 String, c93 String, c94 String, c95 String, c96 String, c97 String, c98 String, c99 String) ENGINE = MergeTree ORDER BY id + + SYSTEM STOP MERGES t_insert_sparse + + INSERT INTO FUNCTION file('test_data_sparse.json', LineAsString) SELECT format('{{ "id": {}, "c{}": "{}" }}', number, number % 50, hex(rand())) FROM numbers(100000) SETTINGS engine_file_truncate_on_insert = 1 + + INSERT INTO t_insert_sparse SELECT * FROM file('test_data_sparse.json', JSONEachRow) + + INSERT INTO t_insert_sparse SELECT * FROM file('test_data_sparse.json', JSONEachRow) + + DROP TABLE IF EXISTS t_insert_sparse + From 1b1d2d67d7b8d06af527eaca236306b7311b0cd6 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 23 Sep 2024 22:55:27 +0000 Subject: [PATCH 11/65] fix perf test --- tests/performance/insert_sparse_column.xml | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tests/performance/insert_sparse_column.xml b/tests/performance/insert_sparse_column.xml index acc94e418f5..0f6cdcec332 100644 --- a/tests/performance/insert_sparse_column.xml +++ b/tests/performance/insert_sparse_column.xml @@ -3,7 +3,11 @@ SYSTEM STOP MERGES t_insert_sparse - INSERT INTO FUNCTION file('test_data_sparse.json', LineAsString) SELECT format('{{ "id": {}, "c{}": "{}" }}', number, number % 50, hex(rand())) FROM numbers(100000) SETTINGS engine_file_truncate_on_insert = 1 + + INSERT INTO FUNCTION file('test_data_sparse.json', LineAsString) + SELECT '{{"id": ' || number || ', "c' || number % 50 || '": "' || hex(rand()) || '"}}' + FROM numbers(100000) SETTINGS engine_file_truncate_on_insert = 1 + INSERT INTO t_insert_sparse SELECT * FROM file('test_data_sparse.json', JSONEachRow) From 7835b66565b481c61e535d2baee362736d2b750a Mon Sep 17 00:00:00 2001 From: 1on Date: Thu, 26 Sep 2024 19:10:20 +0300 Subject: [PATCH 12/65] Fixed MaterializedPostgreSQL for subset of columns for several similar table names --- .../PostgreSQLReplicationHandler.cpp | 19 +++--- .../test.py | 63 ++++++++++++++----- 2 files changed, 60 insertions(+), 22 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 2fe1fb5905a..9ccd0ee778a 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -723,16 +723,21 @@ Strings PostgreSQLReplicationHandler::getTableAllowedColumns(const std::string & if (tables_list.empty()) return result; - size_t table_pos = tables_list.find(table_name); - if (table_pos == std::string::npos) + size_t table_pos = 0; + while (true) { - return result; + table_pos = tables_list.find(table_name, table_pos + 1); + if (table_pos == std::string::npos) + return result; + if (table_pos + table_name.length() + 1 > tables_list.length()) + return result; + if (tables_list[table_pos + table_name.length() + 1] == '(' || + tables_list[table_pos + table_name.length() + 1] == ',' || + tables_list[table_pos + table_name.length() + 1] == ' ' + ) + break; } - if (table_pos + table_name.length() + 1 > tables_list.length()) - { - return result; - } String column_list = tables_list.substr(table_pos + table_name.length() + 1); column_list.erase(std::remove(column_list.begin(), column_list.end(), '"'), column_list.end()); boost::trim(column_list); diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index e64c9eb9d1e..140250508be 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -1184,64 +1184,97 @@ def test_partial_table(started_cluster): ) -def test_partial_and_full_table(started_cluster): +# it is important to check case when table name, with subset of columns, is substring of table with full set of columns +@pytest.mark.parametrize( + "table_list", + [ + "{}(key, x, z), {}_full, {}_full1", + "{}_full, {}(key, x, z), {}_full1", + "{}_full,{}(key, x, z),{}_full1", + "{}_full,{}_full1,{}(key, x, z)", + ], +) +def test_partial_and_full_table(started_cluster, table_list): table = "test_partial_and_full_table" + table_list = table_list.format(table, table, table) + print(table_list) pg_manager.create_postgres_table( table, "", - f"""CREATE TABLE {table}1 ( + f"""CREATE TABLE {table} ( key integer PRIMARY KEY, x integer DEFAULT 0, y integer, z text DEFAULT 'z'); """, ) - pg_manager.execute(f"insert into {table}1 (key, x, y, z) values (1,1,1,'1');") - pg_manager.execute(f"insert into {table}1 (key, x, y, z) values (2,2,2,'2');") + pg_manager.execute(f"insert into {table} (key, x, y, z) values (1,1,1,'1');") + pg_manager.execute(f"insert into {table} (key, x, y, z) values (2,2,2,'2');") + pg_manager.create_postgres_table( - table, + table + "_full", "", - f"""CREATE TABLE {table}2 ( + f"""CREATE TABLE {table}_full ( key integer PRIMARY KEY, x integer DEFAULT 0, y integer, z text DEFAULT 'z'); """, ) - pg_manager.execute(f"insert into {table}2 (key, x, y, z) values (3,3,3,'3');") - pg_manager.execute(f"insert into {table}2 (key, x, y, z) values (4,4,4,'4');") + pg_manager.execute(f"insert into {table}_full (key, x, y, z) values (3,3,3,'3');") + pg_manager.execute(f"insert into {table}_full (key, x, y, z) values (4,4,4,'4');") + + pg_manager.create_postgres_table( + table + "_full1", + "", + f"""CREATE TABLE {table}_full1 ( + key integer PRIMARY KEY, + x integer DEFAULT 0, + y integer, + z text DEFAULT 'z'); + """, + ) + pg_manager.execute(f"insert into {table}_full1 (key, x, y, z) values (5,5,5,'5');") + pg_manager.execute(f"insert into {table}_full1 (key, x, y, z) values (6,6,6,'6');") pg_manager.create_materialized_db( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, settings=[ - f"materialized_postgresql_tables_list = '{table}1(key, x, z), {table}2'", + f"materialized_postgresql_tables_list = '{table_list}'", "materialized_postgresql_backoff_min_ms = 100", "materialized_postgresql_backoff_max_ms = 100", ], ) check_tables_are_synchronized( instance, - f"{table}1", + f"{table}", postgres_database=pg_manager.get_default_database(), columns=["key", "x", "z"], ) check_tables_are_synchronized( - instance, f"{table}2", postgres_database=pg_manager.get_default_database() + instance, f"{table}_full", postgres_database=pg_manager.get_default_database() + ) + check_tables_are_synchronized( + instance, f"{table}_full1", postgres_database=pg_manager.get_default_database() ) - pg_manager.execute(f"insert into {table}1 (key, x, z) values (3,3,'3');") - pg_manager.execute(f"insert into {table}2 (key, x, z) values (5,5,'5');") + pg_manager.execute(f"insert into {table} (key, x, z) values (7,7,'7');") + pg_manager.execute(f"insert into {table}_full (key, x, z) values (8,8,'8');") + pg_manager.execute(f"insert into {table}_full1 (key, x, z) values (9,9,'9');") check_tables_are_synchronized( instance, - f"{table}1", + f"{table}", postgres_database=pg_manager.get_default_database(), columns=["key", "x", "z"], ) check_tables_are_synchronized( - instance, f"{table}2", postgres_database=pg_manager.get_default_database() + instance, f"{table}_full", postgres_database=pg_manager.get_default_database() + ) + check_tables_are_synchronized( + instance, f"{table}_full1", postgres_database=pg_manager.get_default_database() ) From 7855361db5643b9dcc13eb702312e3f3d242c2da Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 30 Sep 2024 12:17:12 +0000 Subject: [PATCH 13/65] Check for Nullable(Nothing) type during ALTER TABLE MODIFY COLUMN/QUERY --- src/Storages/AlterCommands.cpp | 3 +++ src/Storages/StorageFactory.cpp | 4 ++-- src/Storages/StorageFactory.h | 2 ++ src/Storages/StorageMaterializedView.cpp | 4 ++++ .../03243_check_for_nullable_nothing_in_alter.reference | 0 .../03243_check_for_nullable_nothing_in_alter.sql | 9 +++++++++ 6 files changed, 20 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03243_check_for_nullable_nothing_in_alter.reference create mode 100644 tests/queries/0_stateless/03243_check_for_nullable_nothing_in_alter.sql diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 460d74e68bf..7b402464562 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -35,6 +35,7 @@ #include #include #include +#include #include #include #include @@ -1365,6 +1366,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const "Data type have to be specified for column {} to add", backQuote(column_name)); validateDataType(command.data_type, DataTypeValidationSettings(context->getSettingsRef())); + checkAllTypesAreAllowedInTable(NamesAndTypesList{{command.column_name, command.data_type}}); /// FIXME: Adding a new column of type Object(JSON) is broken. /// Looks like there is something around default expression for this column (method `getDefault` is not implemented for the data type Object). @@ -1453,6 +1455,7 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const if (command.data_type) { validateDataType(command.data_type, DataTypeValidationSettings(context->getSettingsRef())); + checkAllTypesAreAllowedInTable(NamesAndTypesList{{command.column_name, command.data_type}}); const GetColumnsOptions options(GetColumnsOptions::All); const auto old_data_type = all_columns.getColumn(options, column_name).type; diff --git a/src/Storages/StorageFactory.cpp b/src/Storages/StorageFactory.cpp index fe7ef5aed54..a3579df6c6a 100644 --- a/src/Storages/StorageFactory.cpp +++ b/src/Storages/StorageFactory.cpp @@ -28,11 +28,11 @@ namespace ErrorCodes /// Some types are only for intermediate values of expressions and cannot be used in tables. -static void checkAllTypesAreAllowedInTable(const NamesAndTypesList & names_and_types) +void checkAllTypesAreAllowedInTable(const NamesAndTypesList & names_and_types) { for (const auto & elem : names_and_types) if (elem.type->cannotBeStoredInTables()) - throw Exception(ErrorCodes::DATA_TYPE_CANNOT_BE_USED_IN_TABLES, "Data type {} cannot be used in tables", elem.type->getName()); + throw Exception(ErrorCodes::DATA_TYPE_CANNOT_BE_USED_IN_TABLES, "Data type {} of column '{}' cannot be used in tables", elem.type->getName(), elem.name); } diff --git a/src/Storages/StorageFactory.h b/src/Storages/StorageFactory.h index f3603419651..c719aed7466 100644 --- a/src/Storages/StorageFactory.h +++ b/src/Storages/StorageFactory.h @@ -135,4 +135,6 @@ private: Storages storages; }; +void checkAllTypesAreAllowedInTable(const NamesAndTypesList & names_and_types); + } diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 2dee120ba13..435c8db377f 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -517,6 +517,10 @@ void StorageMaterializedView::alter( /// We need to copy the target table's columns (after checkTargetTableHasQueryOutputColumns() they can be still different - e.g. the data types of those columns can differ). new_metadata.columns = target_table_metadata->columns; } + else + { + checkAllTypesAreAllowedInTable(new_metadata.getColumns().getAll()); + } DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata); setInMemoryMetadata(new_metadata); diff --git a/tests/queries/0_stateless/03243_check_for_nullable_nothing_in_alter.reference b/tests/queries/0_stateless/03243_check_for_nullable_nothing_in_alter.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03243_check_for_nullable_nothing_in_alter.sql b/tests/queries/0_stateless/03243_check_for_nullable_nothing_in_alter.sql new file mode 100644 index 00000000000..42dda5aba0e --- /dev/null +++ b/tests/queries/0_stateless/03243_check_for_nullable_nothing_in_alter.sql @@ -0,0 +1,9 @@ +create table src (x Nullable(Int32)) engine=Memory; +alter table src modify column x Nullable(Nothing); -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_TABLES} +create table dst (x Nullable(Int32)) engine=Memory; +create materialized view v to dst as select x from src; +alter table v modify query select NULL as x from src; -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_TABLES} +drop view v; +drop table dst; +drop table src; + From 0666792cbe50ba05e56bae080de2a65ff887c721 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 30 Sep 2024 12:23:25 +0000 Subject: [PATCH 14/65] Update test --- .../0_stateless/03243_check_for_nullable_nothing_in_alter.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/03243_check_for_nullable_nothing_in_alter.sql b/tests/queries/0_stateless/03243_check_for_nullable_nothing_in_alter.sql index 42dda5aba0e..c77989ef8a6 100644 --- a/tests/queries/0_stateless/03243_check_for_nullable_nothing_in_alter.sql +++ b/tests/queries/0_stateless/03243_check_for_nullable_nothing_in_alter.sql @@ -1,3 +1,6 @@ +drop table if exists src; +drop table if exists dst; +drop view if exists v; create table src (x Nullable(Int32)) engine=Memory; alter table src modify column x Nullable(Nothing); -- {serverError DATA_TYPE_CANNOT_BE_USED_IN_TABLES} create table dst (x Nullable(Int32)) engine=Memory; From 45309576031416ee371ea23deb07272fa4cb55b4 Mon Sep 17 00:00:00 2001 From: marco-vb Date: Mon, 30 Sep 2024 14:17:13 +0100 Subject: [PATCH 15/65] Address requested changes, improve documentation and fix code-style. --- docs/en/operations/settings/merge-tree-settings.md | 4 +++- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 2 +- .../test_stop_insert_when_disk_close_to_full/test.py | 3 ++- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index d2df5123af0..8a106720ee0 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -1090,4 +1090,6 @@ Possible values: - Float, 0.0 - 1.0 -Default value: 0.0 \ No newline at end of file +Default value: 0.0 + +Note that if both `min_free_disk_ratio_to_perform_insert` and `min_free_disk_bytes_to_perform_insert` are specified, ClickHouse will count on the value that will allow to perform inserts on a bigger amount of free memory. diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 7592ae624f3..88cf4181445 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -580,7 +580,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( { const auto & disk = data_part_volume->getDisk(); const UInt64 & total_disk_bytes = disk->getTotalSpace().value_or(0); - const UInt64 & free_disk_bytes = disk->getAvailableSpace().value_or(0.0); + const UInt64 & free_disk_bytes = disk->getAvailableSpace().value_or(0); const UInt64 & min_bytes_from_ratio = static_cast(min_ratio_to_perform_insert * total_disk_bytes); const UInt64 & needed_free_bytes = std::max(min_bytes_to_perform_insert, min_bytes_from_ratio); diff --git a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py index 82c36adda51..b7b904ec20a 100644 --- a/tests/integration/test_stop_insert_when_disk_close_to_full/test.py +++ b/tests/integration/test_stop_insert_when_disk_close_to_full/test.py @@ -1,6 +1,7 @@ import pytest -from helpers.cluster import ClickHouseCluster, ClickHouseInstance + from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster, ClickHouseInstance cluster = ClickHouseCluster(__file__) From a7eb7a9208cd9a471cc2f6a363636ac53fc7bfa9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 30 Sep 2024 17:09:29 +0200 Subject: [PATCH 16/65] Fix wrong result with skipping index --- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 3 +++ .../03246_skipping_index_70108.reference | 2 ++ .../0_stateless/03246_skipping_index_70108.sh | 19 ++++++++++++++++++ .../0_stateless/data_i70108/repro.tsv.zstd | Bin 0 -> 16286 bytes 4 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/03246_skipping_index_70108.reference create mode 100755 tests/queries/0_stateless/03246_skipping_index_70108.sh create mode 100644 tests/queries/0_stateless/data_i70108/repro.tsv.zstd diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index fa242fccbc1..497520a94f2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -100,7 +100,10 @@ void MergeTreeIndexGranuleSet::deserializeBinary(ReadBuffer & istr, MergeTreeInd size_t rows_to_read = field_rows.safeGet(); if (rows_to_read == 0) + { + block.clear(); return; + } size_t num_columns = block.columns(); diff --git a/tests/queries/0_stateless/03246_skipping_index_70108.reference b/tests/queries/0_stateless/03246_skipping_index_70108.reference new file mode 100644 index 00000000000..463ab3429c6 --- /dev/null +++ b/tests/queries/0_stateless/03246_skipping_index_70108.reference @@ -0,0 +1,2 @@ +22 +22 diff --git a/tests/queries/0_stateless/03246_skipping_index_70108.sh b/tests/queries/0_stateless/03246_skipping_index_70108.sh new file mode 100755 index 00000000000..1a5a6631825 --- /dev/null +++ b/tests/queries/0_stateless/03246_skipping_index_70108.sh @@ -0,0 +1,19 @@ +#!/usr/bin/env bash +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -m " + CREATE TABLE t + ( + c0 UInt64, + c1 String, + INDEX c1_idx c1 TYPE set(666) GRANULARITY 1 + ) + ENGINE = MergeTree + ORDER BY c0; + INSERT INTO t SELECT * FROM file('$CURDIR/data_i70108/repro.tsv.zstd'); + +SELECT count() FROM t WHERE c1 = 'dedenk1d4q' SETTINGS use_skip_indexes=1; +SELECT count() FROM t WHERE c1 = 'dedenk1d4q' SETTINGS use_skip_indexes=0; +" diff --git a/tests/queries/0_stateless/data_i70108/repro.tsv.zstd b/tests/queries/0_stateless/data_i70108/repro.tsv.zstd new file mode 100644 index 0000000000000000000000000000000000000000..1690a8339cc2a7bac1e9bd893b966eefd2796202 GIT binary patch literal 16286 zcmcI~2Ut^Gwsu1A1PCG0gwR{01OxC&4hNKvFHQl)nk z5tLp83xWtL>c4}|y)$#a`R{x)b2t0h2`BrkUDn#`eb+ii_`$L7C5>;PkRWEr=aW9P zH3D4PIdBmHh8Mb?5D&-&$hZx)0dMahvHbk`^Wsc{=R?n#1{AR{5evU|%5sC}jG%T3 z|ErtxaO`2v1JD&*yaalS5qmNdd#a`IYhM6Yz{jD`t!lDzN#E09ru(ztMx_rVZ8n}W zaWSB15K9t}40`8)4?oa}DlZF^FF-ID{v8l*LF~j&lpY4U%z`W6qojp77rXw?%ofhb zK+lYl6c>(wVPrtr0g}ORGeCM3aSmEU@nm3CP2dXn(7r4nA1O)fDFb>w0T*9^-v4;I z`F}HA`VJn2xX}8MO-HE<6CIiv=CRmGescW)FV`eksGS-w*-(%YR~B+ulg0VRx5rAu zAFk~0H@JIe=B>=X*=!gWZmBHfW7Ea)M@T1=Wd+EyGc(0UqPn&)x^_vN-EGrr{&mQQ zZg#~3l^VR~Q^Ewv!W}D6TE0WjMHeKg^gb#6MATssq^GAJI0ycrr{`e*UF0yXzsieI zVksyrTzJi7apX#dzOvo&p_jkKsG!kX{p;T_g#?tcwGeA3~2&?b;!4MqUTdI zi(uwp_7_dS8rJil@)o8eIeBkM&Qp-kj=obQ=*CYIs*=axQk<0>8`@+knilJ* z!B4il;3|oTqmHGD2`^MdQc;kRK$iEeUYlCmmOp54zmndwD7o){=8}2&xbP>8t`9uA zgJ)|;@xp}j-%gdJ+$yD59!?ROKjobxm)dQcVQ!p?VUAQiY8YKVHrU(Mh8Jh;*N{dr zlWcookhI+q8f0DVMm%(6q$E(tx9@p#yG=?APu^wct<=iXU3-;o^XxW;MtLZOv%64| z>fq6hnfqDd6fJk-vP+VM$-V|C<|a663R0}v-!>9VVM>N2MHjbQ3Jdb1co?Ec?0Aqc zD#+-Ey@TqJ74Ln|sZZjG-F91>4R0$AN{!du(wBu9BGu;C{AbICNUGmh<^5Aj{9mDO97G(`V21P}x#{ z#Ao))wZ;%|zzSe9udjZ1y!Sf1-BuDwM-kq>(OkU=YId31VO=OX(Na>M6Fdee2m-5MeIrZkD6 z$9lNbh?(+px*JQ%Ye`BnE{G@u0|8qN_WS)GmJ4=GQO-Mq5S|d#0$ba8qFN4h^FJF3 zyACQ85}Ra%mA2B1qB0JPF=BnTk|*2xqV&&)4~;ERG`nId9Ra!)WsvctuUEagL{ zD`2qK&RaK0d9DGAkxg|&eG!1YDXr}HYh=|p0%7-jLBtGgjX23R1!xPT5n#q=OHK;l z;jSymNJ~j_4Qu+=#>Yg2mrsu0ysN!X+JX6%u`4d$&HA<2Ln zy!KiT5f0p-&2ud{iTNb|_aAufXy7*I{-Z+o0GK>lyZpTYfJtMS%ad!&gs$I9~cA+dVRTETj_)9vHTP7%eKTNa};Xg z;O1a&XM5Vh+$;lN54pdinFdB%9jm4a7{G^m&Q1XQcZ!i=$K_-(i9|c_wKu`m3G<A0)2hFyyo2UGPOd7?`Z68y*8Vf&{h?Q)1Kzrms9PnXI+UH*c& zhp7AuoB!3eNI334RsY5a$+iF;-4KqNPtd0yS|SqV+|0Bj@qXYedLwb<-3!0C4QRK1 zagF~(TmFl^{5$P$0_66aH2mT}|6cpsF#g&|{>*oZKg9Syyr?p=qx2uDzsT2rq&?(x zznI;>+tN*q1;5V4bIxX29hojmh>Z-ju`)Ndd7fqlpG-Dp8sn#@CWpSdr2k~Zhcl|L zf{$nZKBUo zv%7nl@R=gcA76gL8N!)Dg-JWns4aO@{$p{>_^I;3Ig%hYlUSyusNGv!pFn zemLYPRg*1=NvsFl0BXt{;b<%f{G_TV)`;_|Jr+ge-iK0tlhc3GRt|5!{HKfgpDut( zDjw2F()()-0Anp!qE@}-fLpcI2?M8`7=dfysNy;;^|UlXgkgUZP5j|$T9MYO9yX>6 zeIWVLcSa9Edgl*&{l2L1_l_^|mq(o26YrV-dqLE|-{*jYgQS3(7Q%^$e6el0|V8em^4U=KX3yuu6)MDQz8g^*E%g+fUdi4n{{t||Ci#Yhr&yZ>bm z$tK2KwS=jRxXS+Y0^w8>82t&%cR;%Tru@kj9p3w=fvx2*X8YHhzxQQ^f4B`b*#9nS ze^^p6?94PIX3=lp z1duy|;r=4;|I9+den;NFqTK&(|4CIJ#?8N?XJQ-umnismF#JD1RQ~Rm{eMr#@7swM z^w*Uv2}&+Kue4vSNdHM%r!UpDceF71)zbaH8lEU$7$_H_0v&VaChfzq&)m^x#p=b|AdGNq96slop}(d-hV(TJ{2 zVqD9_yZTGYf7`=h667!c&Rqh@9QgpzU}8Seb=@c+TRdC>`0_EJ@^ImdSFM`CX z1VWD4#sa5rBJ7G?ybO|>IE|Pi5({db5!5V#gKIlSvJ%Td#1zKClnipXR<`L8WLX2( zLB>{@TLp1AHDTM^pF$5;8qE+1f`uiccnSum{3)Me1T9A_`VRE`c*L*S{nAO;8=_Le zun{mv&DFyMai%06icPn)RF9um?dkR$2(A*N4I9BWT3t!+YO$5FZs}sg zrPuymI?LU4Vxk$V<`8?BIYnXh#FdmVG1B6~UF!R%q={);H|+WNZl78+;T--j9^Wq? zG7b>WmKJ0}<3P6gvI3{l&DIM82Inz;!Ux1Ot=GKS=RmH9ZcLpu-l}6xyOSip?8#5) zaPIACTT&rTr)Aj{;d##5+cJNph8fMG0T6T6A!xFbK>D6e0_ata32|89s0hidg96UAF&bWPwm+Yi|BbTY^NVol>7G zK{H%R1Q=vnXauE9G({A-G4Jf?NN?^f1{XR!3yZT}p^sm=t6y=NsqDyp`Y8LQ33J7h>3i8s;(1p_7>*_B;zOh5RGLXHs_)gO zzu*)XhCutrDoVe3hZ?gONDB2Oj(H5%?H1h7xXnAd5%~3;H6P@j+za($Wv>y5II5X| z)B;!BWSCx+W^U-g3AHm+W$z*yACk8Um!vMpUimc4ktJyj*Gm(;Ui~%N@dl&#qk`L( z?Ht$n=RNWs)E<#{GGIZT4lNS!6LeVe{7%;+d6_iP&F`T@-p;(tj4|esu71(C!-A=wcpBuDfTGHhPc@*sYwsU>4*D0#+MBQLVZ?CSZvDMVcBjBE|^1?Oj`Af?8GM!&4 zIjy+0Bs){dzpd%*PJh{$NPhoC1b&J{=Qbku+rZC{0;`RL2pUQwhRu28;E}UKairX0 zeX6^5m#x;i-mH5d^uIDS^j*!JvmNMt|GcGEy@E63YbOu-JS_)?c?2c88)Jp2%pHIDpQ*my zKeH8Bvb5) zqfwq^`=>0lpTh-zh!H9r8!ry~=`HT*&G`gxoyeMToQx2k`Jm_KHRKYmwa;_yj$Uy5 z#6@LsEUX<8W~enf!|^IlcMv!IjT8`Je@Z6Dv&!4~5GV$+Y^ci;Z9zn$#QyMk|2_)Q?R?Io;c zP%Eezl<25P0P#zR8V1f?yl)G+Ydf#^d7A5CY;8^Q;qzv!+M#c+HKHTpN}yNfamfN6 z>=r#|3n=dJ!4yAq!BSsM$LGQ)Ilu{W)DbA$TY}}OW8SV zgb_+Rcrw>{)s!-RI*)v~q7>v{9QaLES*)Yo^v$N~G!dQ@)8_B%_)La}%1p0aniYK7 zp((vxCwdLg7&QYeB_$#w(QrJ#O1(1=tjE{hXcszjK#TE0(lISqO zE&zWv!YHhyJnwa{(Y9m4F)$Jr&95K*ub$}=kcqwxEN|?ZUh2}fS02p zvXZlQiG%3tWd>^eBKLvBWq@bDrw0P|EyIYUS&c|@-vOE%2nd-HAY`mOtaQ13Do+l1 z>e1MzM2f|(O%vHMW*cJ-R-p$w*$7y|Em&BXjP%iwN1f9ZQg$TqG2l7<`6AUr`JWD| zdANThIz8m7DRTjY59k5V$zi~R0i7hKW8(W%u0I6t+r;&--=aMyRZNl(%d2U4=_x!Jt^-R5MVURgxw$1+5($&wfLi@ z1{y)Yseu)Ix?Dgc+uRP+lf(#!n~2%!G&$uRspOM@d4&h{k~z7eVD8MivK?YNz`MX|~Qpnwf!0EA_y*4Ryd=Q*% zCOL}dTw|_16D{8qX*Ehd5=lkF!W60$#?c-LXJWDISmnK}Zu>M+l?h8F#VEKF+cvQv zjv13=@g{~qS@FLjpk<&Gu%!PAuO9pguYgf4BdNNG&Itq(qxN>4W4-N6fD>AhS%fkI29CFMSa?IFAr%YjzG%<2mpuzz4J8Bb(g}sd ztML?6@#v zQK~H;U$axYSS=)Fj<7OV)3Fj$!5b;3-XCe*vGmo}7k8?YKbJ>lvu}K)i!)U<9eSD$DIfc41?g;OoGBWLn`%DmR;M@K& zQ_XvcC(nG28t*YHF*0rZF+rE(QQlC0#7js!C2vOyxD1WfPwGytEEy@8R~(jX(~gcD zz5X+ZVtmczZvNoprvLKSU~-1biurI~GRX-i^v*+kZ$3tJkAS4}RSdh znq!+NlK$W)y-U=|(9Dca{%zTlmK-~S)Zqk5(;Y6&kT9vIqn#Qfci;DU=H8xZGy7rt zp^tGzAQde%2!lqpr8v?wjcDZEx~Cl1l4`?Fc5*rDKq`Lc4);oVWje&&0bob;H7_>q#V3rho9 z$~$R}i5k7?X(!x2usx4C{@Oh|mZ>;pCt26|xx?-bVKGa`?FDC zsFJGwAMzi`4*HgM4b3*lzUuOZb!nJpz2n&)gk+@pvaEEsCB;GSB2*vBEWHhypQSJ^>q)fXJ9$vq+zNpp7P&!FxRwzi5{B;<9(HhT=|_7;a&EY^3H7pfa@82YI6D6zS9?W$|m}j+G4|Enq~tP$WwM)olz36V{lBBPJ@(rfgc$Hhpu2^7<*BM z-LlzcK{ujX<7H_1&QB^`@{&1Pnbg+Fh!3UXhJS zSC|cj^YwmnQ{bSesnNs&aTC>_s|F_e&{di(L+p*c2|ToOm3L9v5y8xh+o?;lxiBO< zg_p_dRd*~5(Jfp~4#zvNkLtl7rDDpL5OM{MGukNaAbd39Byw^Sf!c3cc9tOeE4qmq0*-h>b_R7Kez`b zYV~wXzbaWWdti2gqA~ON=9p6Eu@8tx!C&4d953Oechgee@TpT}ztFf4#Zf(w)n1kU z(rr#>vq56UwU}P+h3zr4SdBw?g6wlkhlFP~O_p!YYXGIZDmIR{hZiHeg>{VMz*kr} zvWtgeOu$ENk%`eEAj^pH#Ijc>^11d}?#UJC-g1-7a0+AyXFw}fA{oIe&OV|qWiy-? zVymfbdKO=uScfcVYUC&NaZGw(B%A%Q&NlQ!)DaL|EMhWX83fy#ZgCmeQ!7Ol{1~=6 z-MdFlwU0W>)n^UqIirsgtbl4C`F$XVvYi{#l4Q#oh`f7nb*Zg;W$@}V$k2&5v$!5v zEs6W1SXznuUhUrPtV1MC9v8}J26ao`(xk22EGEIJtB`kfrD}(gYg>*@28pA42!UfR z?f8TAb1?DuDC!d5+Qu}i7{Y493#WA25sB=n#RhQ>gg;a1Ey4i#JJzbzdY6oyh8Kj@ zMQOQq6y6Ma;vUQlkFLaFe0k3|6(ELs?lk(QOnkL(K)ee28M0q!fmXrQHCf{ON!+vs z1SnUl8}W@ePup(|7Y1;puETF)1{9NQYH+l^-c6Q7sn&Q*k-2|?q5@k6fhG>u)s!hO z9!qOK<{^L7cm0SD3BeASbKcsbcs9XFX8~*JIMQ7zgl~*13le#JHN(T!7~#Eox6w~K z!=vxgto+gLB_*F(qyxE#eETU?PLHH^Tup@>5S>yRgYZJ5MUZaAtV5k1G9T~O4{ASbHg=0IXAlh`Mw3Gv@=3!^NBzbSLrs;c zgSJjh^KlmWyR?hLcp*y<*4>a*DetBj)}ii|qM{gkv|q;C5I!0zwa}t3WFNJ{eORQ7 zkBuMjqS+FQntE;A%-bT#Xc4v{A$44rV{}hTaN_7?F51ji*6reoW6XV}P7z=9(*%n~ zBf#O~+d?%I-PXQo)o z<1x!9G&&lgj>~Uqe8hGx608S%P2@gIG@GD%?_6e_qOUjUxpYJ+@Kt)EY$Zkh`Nycv zFB)(8NP!-FX#+STFlQ5lE^asKTbkE>#U< zQ$7q!7=HH11(aai>JPDdPrh#4xrb)v1LkMle@bQCyNB!YtapzwB^$_DV<#FLZykTG ziiwL-QAc_yypOjIarU-qq{WMiMsz%GRAWMOGU3g^LaVvWalVDtcu{o4c@o;8$?jVV z_LZ*hTbH33-@b}ck80VJh?}HYYYAtcSqIx-a=yy#v4F(Gm)7o8uB@b8v1^Ol_?kGx zVE3ygOIDmDu+WO44X~KFIN{8}*viEeXEien8zVWIZKBA~AB{F7TtsIRR@7opSSQp( z<^|@Q{wx45KE|TVL)uvB+jL&-Mi@53{sen7Oor{WQxKDxc{?WK$}3r?AyF7miMlUM z%m8^LaOC(w02C{WrkIQy-KA8W8ZU?%rd*9j2sRm1Myr5=%JQQSc;YE`#? zz0d`l`b7h8P)5);?o~n>gP%{nLVlhG8ntRl5*=|xR~ru3d$H;d=(?b%Le%a6?>$Ou=@jBXe#e#QN6c|< zMQA4H*$Q@aQCyig4}K%(OxH664yVp?ZoPda%o!9@y9!rs^!w7!U+7C1!q5}?J6A$Z9z_iLnOtA#4IIggXEW8;{22Nd7+cS7^nJZX28wMvM zG2Barzkqb!xCQPDiqq`qzF8Qf7YKe`yHUTt=^tcRThjuV@cNR4p1P<8e`hh@s6JESE7dxPS z7DH4TapYj857|3JjR8~Hw|;>H@z@Li{o}IW}63a0+~e$PJF9rY2@x@t}4JE zl?aCg#x&w1;5P@|yPRsBjnSy+tkg&kpT3f|Fzi^61|IzsH#c|ypIdnxAe>F;F7@I? z*PZmOwL!nNAA5vCv-vYg$hgG5NjWPC%-XlfdJrcyg6HJSAb3&THL*pbGYIF$a>H(9 zS|9Q;pMER49^4Kk=f_Pn+r?xLB{sr9yhxfs^y_$O zhWnfWW6KY8US1+d7*uJ|+>w{GjXl3qBFUF5lvx|16SqP;rDN6jdho)G*ba`gPJhD!O+BY6(Iw>`~C4_dl>^wcy z0gJhC(7ai(dYpnp?~IlNR_yJAH|jEXIFi~^H5SSkEwO6#)$_;wd#8E(f;eLQ2ZCO1 zXta)GJ_sM5=kB4jrg;1Of^|ft@d@9A=SfBBuN{>(&7V!5r#vc+hp{-Z4t(L(4zK@a zJj)PFuxxX{=+e=4jqzxSQF>EU;mfrghJmJ5r}0STqm;$9 z9tgUq2lt#cU_V_tA1U(4h2A(xTG+Th$FSmqI;q0exQjJQ-_5jEts*~j`HENIvAbTX z-MH%667=4M%U7$z=IGgsZSqVH8pR^9Z438y<|ZFlUR))DSRT~#gSPeVKkjm z1e3m#%}K%kvFmwN3uI_^&|O12&gH1FJJUnO3C@ZmQ=9xcx?W-@UE-^~pd+pOf%Gm5 zs;SlD_K!-KRrXr^beyu&dK=`>uoIk-8aJ2;TJP2G&X@HYK$rJVwTlj(?IB%sOJ+1k z`=rI|LmnYVow7Ao{E0yVU#&*_?CTtw9VfE=^8vC${K2e|V#=)uZZ>C#4HfxK+Zb@z zc;g!q14>4N8N)g3g3cJewneNat{`v0y_c>F4vyZ0`|-6p;if~4?CC|U<82asuSK1d zS~Xu!tHyumQh9hn@#X?EMOVO$*XHeIyM06J#Z|Bd*)#Srl@$B=2}1!3CNE`;p-g)I zMad=uid1Y2H&99a!X>N($rp;{s*{B8gk!8^FMj-hdq_B$S#f`>F|6dL8t?UO zrOFOBnjULTSVoVv`Xa@w-De|9R!CiySA138B*ff$R_W9wXngwgWJh45 zzAtlY{IM$081?W$H&5mQk|SmjJ`e+N~H)zwTqg55WHZ1Yk# zbaI#Ttm*yyXgl@2WM}@|jl>_A5A>qiN#!QuO;N zoBnh`6D7n>#h1(;5#w)U%!$zEsH!Qp!4tO=w&uk`GwljlLym-XQLx;#75?U-!AQ+b zw{mj+sow*2^j$4(F{yTf_IrA=q^0F2^}f=vr9b2&E{s>dAY>_8x?gay^^i!V3d=Wy zqtb7p+)jV_d5pp&$fM77e_K>S_mK<6J;QrFtMcsw#cX}&;2}c(x&4--5<0Ut`EJ$) z#&ir?XOWm3N$l~&ZyQ~vz0Yn<(Q0mJmh$o939R&`W}gdkR9G75v50ovQ=xXrM$;|W zW3P>Fm1r3mv}*J$C{&)K9eq4E+GyUU#3Q~^FVUvq)8!oZ!YDV80++RAL?9)j%G&_Hf zbMW?%xg7+XsL}rP1WoY9ro`$r(}VY8`oHo?8#OAuGU4VE^^94W7vl-`-_RatSV3kg z6wd@qPYAydNS)R9yo2(AtFOQI!6lu_jK}n-OzlSTGkkf;Y^mc$GF*lCNUyP6_KxKt zv}5mbt-;bRs#%ZKsCY3oNsMn^|{Z4cxXcc+uIVn|fJ< zN&eCoh}B~iL(@`!*3g?(KNOi$Z+*3(%uS3p#>Ms3I;B`4niY<>eoK62r}9ikD8XeX`(7 z&vOW3Qt`6qm1@s}E-@G|pIF1TX^$AD&qltx-;L-h!K6g&h9vMzpIhel^LxUZ+2f_; z`q1&HPJM~9)8$o|vH> zJadl;wG%ScpdX8SEIZ&`%C0@$H~=qce>8lK$4(SA=C56>T3~HGc00d?Pa&#Gojs-R zRaUk}&C_Gaq6HuArL-KcH&PhtcKcQDyhlQ^%aBe#Wa!tLTSVjE-%E~!{M^*8ezifL zgW6E>5NDC%$oHLO=>B>6p?Uz>c3jG2zZZe`#G~lfhBh2yCVL6?TpFRW?e7+X`Sck^ z&jm0Bv);AcKEa#S*%(wT7;=R^%)mYSA9*d)SO(o>10P26xGRk z<#?fL!LtPO6i5=gn^3m2!@FK)x_F+m>bJV(EI#L{r6TWsY#+~o#o2l;ogiPPB;RJs zA}CMMSZmwG(`lcQ8Z`;7sSUsn557PB%Hfe`0* Date: Mon, 30 Sep 2024 17:51:56 +0000 Subject: [PATCH 17/65] Fix data race in ColumnObject/ColumnTuple decompress method --- src/Columns/ColumnTuple.cpp | 8 +++-- src/Columns/ColumnVariant.cpp | 8 +++-- ...03246_json_tuple_decompress_race.reference | 0 .../03246_json_tuple_decompress_race.sql | 33 +++++++++++++++++++ 4 files changed, 43 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03246_json_tuple_decompress_race.reference create mode 100644 tests/queries/0_stateless/03246_json_tuple_decompress_race.sql diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index e741eb51c68..0e26ada64ad 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -770,9 +770,11 @@ ColumnPtr ColumnTuple::compress() const return ColumnCompressed::create(size(), byte_size, [my_compressed = std::move(compressed)]() mutable { - for (auto & column : my_compressed) - column = column->decompress(); - return ColumnTuple::create(my_compressed); + Columns decompressed; + decompressed.reserve(my_compressed.size()); + for (const auto & column : my_compressed) + decompressed.push_back(column->decompress()); + return ColumnTuple::create(decompressed); }); } diff --git a/src/Columns/ColumnVariant.cpp b/src/Columns/ColumnVariant.cpp index c6511695f5c..c2b51a5fcf1 100644 --- a/src/Columns/ColumnVariant.cpp +++ b/src/Columns/ColumnVariant.cpp @@ -1393,9 +1393,11 @@ ColumnPtr ColumnVariant::compress() const return ColumnCompressed::create(size(), byte_size, [my_local_discriminators_compressed = std::move(local_discriminators_compressed), my_offsets_compressed = std::move(offsets_compressed), my_compressed = std::move(compressed), my_local_to_global_discriminators = this->local_to_global_discriminators]() mutable { - for (auto & variant : my_compressed) - variant = variant->decompress(); - return ColumnVariant::create(my_local_discriminators_compressed->decompress(), my_offsets_compressed->decompress(), my_compressed, my_local_to_global_discriminators); + Columns decompressed; + decompressed.reserve(my_compressed.size()); + for (const auto & variant : my_compressed) + decompressed.push_back(variant->decompress()); + return ColumnVariant::create(my_local_discriminators_compressed->decompress(), my_offsets_compressed->decompress(), decompressed, my_local_to_global_discriminators); }); } diff --git a/tests/queries/0_stateless/03246_json_tuple_decompress_race.reference b/tests/queries/0_stateless/03246_json_tuple_decompress_race.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03246_json_tuple_decompress_race.sql b/tests/queries/0_stateless/03246_json_tuple_decompress_race.sql new file mode 100644 index 00000000000..f8d599cf2cb --- /dev/null +++ b/tests/queries/0_stateless/03246_json_tuple_decompress_race.sql @@ -0,0 +1,33 @@ +SET allow_experimental_json_type = 1,type_json_skip_duplicated_paths = 1,allow_simdjson = 1; +DROP TABLE IF EXISTS t0; +DROP TABLE IF EXISTS t1; +CREATE TABLE t0 (c0 JSON, c1 Array(Nullable(Decimal))) ENGINE = Memory(); +CREATE TABLE t1 AS t0; +SET min_compress_block_size = 2654980, cross_join_min_rows_to_compress = 1, insert_null_as_default = 1, check_query_single_value_result = 1, partial_merge_join_optimizations = 1, use_uncompressed_cache = 0; +INSERT INTO t0 (c0, c1) VALUES ('{"c1":[{"c1":"Uw"}]}', [-5205545.0]), ('{"c1":{"c0":{"c2":["j^)7`{"]}}}', [-183841.0, 1232295168.0]), ('{"c2":["Dh X,r<"]}', [-518583.0, -563.0, -2442132402.0, -4991142492.0, -34823.0, -5.0, -6.0]), ('{"c2":"n 8k]wa_V"}', NULL), ('{"c2":{"c0":{"c1":-98}}}', NULL), ('{"c2":36}', [-89513.0, -9306968710.0, 6332325249.0, -48245723.0]), ('{"c2":{}}', [-10755380.0, 60433.0, 4047617.0, -98905.0, -993.0, NULL, 93055562.0, 917699.0, NULL]), ('{"c0":true,"c1":"g=n4"}', [752.0, 1.0, 96.0]), ('{"c3":false}', [NULL, 4039255.0, -88372248.0, 1256111069.0, 2383.0]); +INSERT INTO t1 (c0, c1) VALUES ('{"c2":["7AKy`Zh",[[-94]]],"c0":"&A:2ho","c3":[[{}],{"c2":257}]}', [-7451549.0, 633.0, 7009760932.0, NULL, NULL]), ('{"c2":65}', [-12226902.0]), ('{"c1":[{"c2":{}},"Tk26#E)"],"c3":"","c3":-608}', [-342709852.0, -6359192.0, -13.0, 403.0, 88126338.0, -31192275.0, -5.0, -4840.0, -82845285.0]); +INSERT INTO t0 (c0, c1) VALUES ('{"c0":[":+k,"]}', [-265468152.0, 96787546.0, -8980013521.0, 9164448735.0, -1447.0]), ('{"c0":[[{}]],"c0":{"c0":973,"c0":-345}}', [50.0, NULL, 805297790.0, 3038967.0, 7847438.0, 5716507241.0, 26414475.0]), ('{"c2":["7/jinz|"]}', NULL), ('{"c2":972}', [10507.0, -9616192.0, 595.0, -6102996.0, NULL, -17.0]); +INSERT INTO t1 (c0, c1) VALUES ('{"c3":{},"c2":[true,"US","a.;lxbvH1w"],"c3":{}}', [NULL, -5633149870.0, -1557199.0, -686626.0, NULL, 22726.0, 9771277778.0, -684868.0]), ('{"c3":{}}', [-502.0, 16260784.0, -584.0, 2265729.0, 1.0, 3768742.0]), ('{"c1":-570}', []), ('{"c1":[[312]],"c1":-27}', [60.0, 4.0, -3054.0, -216.0, 85786.0, -16407500.0, 1.0, -64565119.0]), ('{"c2":{"c1":"CM0tk"}}', [-3571176.0, -24519.0, 757451.0]), ('{"c1":767,"c3":421}', [-6116211.0, NULL, -77835774.0]), ('{"c0":805}', [5345543778.0, -6493852.0, 461884.0, -3158812.0, -3.0, 1.0]); +INSERT INTO t0 (c0, c1) VALUES ('{"c1":669,"c1":[{"c3":-101},{"c2":[443]}]}', [NULL, -50848.0, 9270050424.0, 439285082.0, 4991131460.0, 5324167069.0]), ('{"c1":{"c0":654}}', [18184400.0, 1165.0, -9067743190.0, 55008.0, 84573.0, 312777.0, -38.0, -180.0]), (NULL, [-51431.0, -205.0, 6391704.0, -3531393554.0, 4.0, -445378.0, 4499335205.0]), ('{"c2":-973}', [-13697135.0, -3232582571.0, 5063774471.0, -671011.0, 1882007.0, -94.0, -42828350.0, -9.0]), ('{"c0":[{}]}', []), ('{"c0":[961,"FE"],"c2":-74}', [-149626420.0]), ('{"c1":[-936,false]}', [-5436.0, -4267685.0, -9337344399.0, 90404.0, -24037337.0]), ('{"c1":[null,{}]}', [-50821332.0]), ('{"c0":"sC06!j0Y,W"}', [4834282.0, -863431.0, -535818460.0, 9592396.0]); +INSERT INTO t1 (c0, c1) VALUES ('{"c2":null,"c1":"bo^v6"}', [-1719.0, -16074.0, -3.0]), ('{"c0":{"c0":{}}}', [-3826.0, 2.0, 160017.0, 19500513.0, -8.0, -739458.0, NULL, 4420975388.0, -5230674.0]), ('{"c1":{"c2":326,"c3":{"c2":[-66]}}}', [-29.0, 742516.0, -6328.0, NULL, -1.0, 3.0, 877215.0]), ('{"c1":[{"c1":{}}]}', [168872177.0, 48258375.0, -6983476.0, -7633.0, 1.0]), ('{"c3":{"c0":306}}', [-64221197.0, NULL, NULL, -3753326.0, -10665.0]), ('{"c1":[";~R&R2Eb9o","|abDlI``-j"]}', [NULL, 8.0, 697608174.0, 323490017.0]), ('{"c2":[[["y(4:erKU/(",423],[null]],[false,[-137]],-282],"c0":937}', []), ('{"c0":{"c3":true}}', []); +INSERT INTO t1 (c0, c1) VALUES ('{"c2":{}}', [-702.0]), ('{"c2":true}', [8756332921.0, 1128192142.0]), ('{"c3":"iNGbzf","c1":{}}', [-442.0, 1439.0, -58.0, -6321.0, 9803746.0, -98.0]), ('{"c2":{"c0":[true,25],"c3":-887},"c3":true}', []), ('{"c3":[{"c3":[-568,true,""],"c1":{}},[{},{}],{"c2":-755}],"c0":{"c3":{"c3":null,"c3":{}},"c0":{"c3":638}}}', [2.0, 96001085.0]), ('{"c0":{}}', [524.0, NULL, -1252951.0, 1017260.0, -81620.0]), ('{"c3":{}}', [6.0]), ('{"c2":{"c0":{},"c0":{}}}', [-578.0, -6053615.0, -927647.0, 55.0, 29276.0]); +INSERT INTO t1 (c0, c1) VALUES ('{"c1":{"c2":{},"c0":{}},"c1":{"c0":{}}}', NULL), ('{"c2":{"c2":{"c2":[true,null],"c2":{}}}}', NULL), ('{"c0":[null],"c1":{}}', [58.0, 630440989.0, -64846.0, -7344.0, -220570.0, -2.0, -1.0]), ('{"c3":{}}', [-28.0, 113441645.0]), ('{"c0":null}', [-6.0, -80605.0]), ('{"c1":null}', [NULL, 68176530.0]); +INSERT INTO t1 (c0, c1) VALUES ('{"c2":"O(w1RrE","c3":598,"c1":{"c2":{"c2":-476,"c1":{}}}}', []), ('{"c2":{"c1":[{},{},-517],"c1":{"c3":{},"c0":-392,"c3":"A7_a"},"c0":"Lcuchjta"},"c0":-939}', [-592.0, -21901.0, -19.0, -268264638.0, 43.0, -4676673989.0, -9055.0, -44.0, -769.0]), ('{"c2":-500}', []), ('{"c3":["Yf-{*M,Z[b"],"c3":[[[false,true],-23],{"c1":{}}]}', [-4.0, -76.0, -1834.0, 116114.0]), ('{"c1":{"c3":{"c0":{}},"c2":{"c2":{},"c1":[-766,":o;o]B@b 5"],"c2":["[ZL@tVniT😂"]}},"c2":"G","c3":true}', [-424454555.0, 464554127.0, -271.0, -6767008.0]), ('{"c0":719}', [621.0, -640.0]), ('{"c3":"T4Wz"}', [-511.0, -1.0, -83925131.0, 264491.0, -1.0]), ('{"c1":[["i]6yMcs|cB",true],{}]}', [-6.0, 83219.0, -6735.0, 192018793.0, 1956.0, -9573927.0, 84699.0, 54263916.0, 631.0]), ('{"c3":{"c0":74}}', [1206189.0, -7592480392.0, -93161125.0, 817865670.0]); +INSERT INTO t1 (c0, c1) VALUES ('{"c3":[-546]}', [2.0, NULL, -1326235.0]), ('{"c3":672}', []), ('{"c0":{}}', [7.0, 59133.0, -56738201.0, -49026173.0, -81121.0]), ('{"c0":442}', [-8408.0, 691.0, -7.0, -253.0]), ('{"c3":{},"c1":{}}', [4931930.0, -7377.0, 158.0, 36156.0, 803877.0, NULL, NULL, 62.0, -9846.0]), (NULL, [-1758064.0, 4290.0, 4775.0, NULL, 22.0, -439.0]), ('{"c2":["",136],"c0":"ib"}', [2645931.0, -674758808.0, 5014674.0, 76.0, -1355423029.0, -7520419854.0, -6.0, 78413978.0, -4011.0]), ('{"c1":{}}', [NULL, -544875204.0]), ('{"c3":[[170]]}', [NULL, NULL, 73890.0]), ('{"c1":{}}', [183547.0, 93516376.0, 5.0, -720.0, -749201.0, 123727322.0, -65.0]); +INSERT INTO t0 (c0, c1) VALUES ('{"c1":"","c0":274}', NULL), ('{"c0":[":0pN9k*W"]}', [60.0, 25.0, 6.0, 9520.0, 90466666.0, -3.0]); +INSERT INTO t0 (c0, c1) VALUES ('{"c0":{"c3":{"c1":[null]}},"c2":{"c0":92}}', NULL), ('{"c2":{"c2":[true]}}', [NULL, -95331.0, NULL, 1308199.0, NULL]), ('{"c1":[[{}]]}', [276463640.0, 718712799.0, -50123.0, -12043.0, NULL]); +INSERT INTO t0 (c0, c1) VALUES ('{"c3":{"c3":[false]}}', [-260.0, NULL, -1.0, -40.0]), (NULL, []), ('{"c3":{"c3":[{}]}}', [-3117135934.0, 173.0]), ('{"c1":{"c3":[[-112]],"c1":"%nI"}}', [-2510.0]), ('{"c3":{}}', [-638201656.0, NULL, 18.0, 56925070.0, -6815.0, -869.0, -36617736.0]), ('{"c2":["X"]}', NULL), ('{"c0":{},"c3":[null,"*3QZc8",true]}', [-1.0, 84.0, -819479844.0]), ('{"c2":{"c2":true}}', [-107.0, NULL]), ('{"c3":true}', [-278665.0, 116.0, 18.0, 31965.0, 5711148.0, -8234.0, NULL, -19369679.0]); +INSERT INTO t1 (c0, c1) VALUES ('{"c2":[-225]}', [1.0, 1510841132.0, -12.0, 1307.0, -4483.0, 55.0, 9549.0]), ('{"c1":[{"c0":-728}],"c2":{"c2":[-958,{}],"c3":[true]}}', [-4053.0, -876356.0, NULL]), ('{"c3":[[[24]]],"c0":{"c1":401,"c2":{"c3":[483,null],"c1":-83},"c1":{"c1":-203}},"c3":-680}', [NULL]), ('{"c0":{}}', []), ('{"c2":{}}', [-1707859140.0, -5.0]); +INSERT INTO t0 (c0, c1) VALUES ('{"c1":false,"c1":-360,"c3":-739}', [-3868.0, 548174539.0, 78824.0, NULL, 964751.0]), ('{"c3":{"c1":-751}}', [3225.0, -333274171.0]), ('{"c3":[{"c3":false},[{}]],"c1":{"c1":{"c0":{},"c0":{},"c3":true},"c2":{"c1":{}},"c0":16},"c3":{"c1":[{},577,{}],"c0":{"c1":false,"c2":{},"c3":{}},"c1":{"c0":{},"c1":{},"c0":"Q9}f*"}}}', [-146.0, NULL, -1984141.0, -5535507413.0]), ('{"c1":[[["CYa"]]],"c1":[[892]],"c1":{"c0":null}}', []), ('{"c1":["__C`X ;Oy4"]}', [68158746.0, -173.0, 12.0, -5.0, -8881621.0, 1822742.0, 752262442.0, -97340.0]), ('{"c3":935,"c2":[-999]}', [-212414562.0]), ('{"c1":false}', []); +INSERT INTO t0 (c0, c1) VALUES ('{"c3":"w=v%C"}', [-133239.0, 41893484.0]), ('{"c0":"?D.B#["}', [-660565014.0, -3.0, 1778026873.0, -12892.0, 37295.0, -8.0, -4049.0]); +INSERT INTO t1 (c0, c1) VALUES ('{"c0":96}', [-9486418055.0, 1.0, -19153.0, -3330.0]), ('{"c1":{"c1":""}}', [-120.0]), ('{"c3":{"c2":[false,"sT"],"c2":{}},"c2":{"c0":[null],"c2":["t0-}.Dm",119]}}', [4008130576.0, -6381371.0, 660095684.0, -892497.0, -76.0, -811584704.0, NULL, 16359874.0, -315983.0]), ('{"c3":{"c1":{"c0":{}}}}', [-1086.0]), ('{"c2":{"c3":"f"}}', [3064910.0]), (NULL, [-51357.0, 8319955.0]), ('{"c1":false}', [NULL, 12020.0, 44851173.0, 89.0]), ('{"c3":[false]}', [4.0, -361122.0]), ('{"c2":[[975]],"c1":[505]}', [-833.0]), ('{"c0":{"c3":["e~"]}}', []); +INSERT INTO t0 (c0, c1) VALUES ('{"c0":{}}', [-2664277.0, NULL]), ('{"c3":{}}', [490623582.0, 2.0, -77004.0, -1101.0, -1573.0, 5.0]), ('{"c2":"!Xs.wZ{>^B"}', [8571380046.0, -27.0, 1.0, -29.0, -45787234.0]), ('{"c3":{}}', NULL), ('{"c3":[[{}],"n<>M9w9"],"c3":[[[-565]],true]}', [-4037092.0, -27.0, NULL, 8364633.0, 120211.0, -800861.0, -3.0, 656171602.0, 1480.0]); +INSERT INTO t1 (c0, c1) VALUES ('{"c2":"O Date: Mon, 30 Sep 2024 19:29:53 +0000 Subject: [PATCH 18/65] Fix possible hung in ALTER COLUMN with Dynamic type --- src/Interpreters/MutationsInterpreter.cpp | 21 +++++++++++++------ .../03246_alter_update_dynamic_hung.reference | 0 .../03246_alter_update_dynamic_hung.sql | 7 +++++++ 3 files changed, 22 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03246_alter_update_dynamic_hung.reference create mode 100644 tests/queries/0_stateless/03246_alter_update_dynamic_hung.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 22ccfdd5f4f..14ec539036a 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -513,12 +513,6 @@ static void validateUpdateColumns( throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "There is no column {} in table", backQuote(column_name)); } } - else if (storage_columns.getColumn(GetColumnsOptions::Ordinary, column_name).type->hasDynamicSubcolumns()) - { - throw Exception(ErrorCodes::CANNOT_UPDATE_COLUMN, - "Cannot update column {} with type {}: updates of columns with dynamic subcolumns are not supported", - backQuote(column_name), storage_columns.getColumn(GetColumnsOptions::Ordinary, column_name).type->getName()); - } } } @@ -1365,6 +1359,21 @@ void MutationsInterpreter::validate() } } + const auto & storage_columns = source.getStorageSnapshot(metadata_snapshot, context)->metadata->getColumns(); + for (const auto & command : commands) + { + for (const auto & [column_name, _] : command.column_to_update_expression) + { + auto column = storage_columns.tryGetColumn(GetColumnsOptions::Ordinary, column_name); + if (column && column->type->hasDynamicSubcolumns()) + { + throw Exception(ErrorCodes::CANNOT_UPDATE_COLUMN, + "Cannot update column {} with type {}: updates of columns with dynamic subcolumns are not supported", + backQuote(column_name), storage_columns.getColumn(GetColumnsOptions::Ordinary, column_name).type->getName()); + } + } + } + QueryPlan plan; initQueryPlan(stages.front(), plan); diff --git a/tests/queries/0_stateless/03246_alter_update_dynamic_hung.reference b/tests/queries/0_stateless/03246_alter_update_dynamic_hung.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03246_alter_update_dynamic_hung.sql b/tests/queries/0_stateless/03246_alter_update_dynamic_hung.sql new file mode 100644 index 00000000000..e3bf7bb5a37 --- /dev/null +++ b/tests/queries/0_stateless/03246_alter_update_dynamic_hung.sql @@ -0,0 +1,7 @@ +SET allow_experimental_dynamic_type = 1; +DROP TABLE IF EXISTS t0; +CREATE TABLE t0 (c0 Int) ENGINE = MergeTree() ORDER BY tuple(); +INSERT INTO t0 (c0) VALUES (1); +ALTER TABLE t0 UPDATE c0 = EXISTS (SELECT 1 FROM t1 CROSS JOIN t0) WHERE 1; +ALTER TABLE t0 MODIFY COLUMN c0 Dynamic; --{serverError UNFINISHED} +DROP TABLE t0; From 3075e2c460ebe58c88b301ce8a5f4b3496d2ab63 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 30 Sep 2024 20:12:21 +0000 Subject: [PATCH 19/65] Fix bcrypt password being displayed in system.query_log --- src/Parsers/Access/ASTAuthenticationData.cpp | 3 ++- tests/queries/0_stateless/01292_create_user.reference | 1 + tests/queries/0_stateless/01292_create_user.sql | 11 +++++++++++ 3 files changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Parsers/Access/ASTAuthenticationData.cpp b/src/Parsers/Access/ASTAuthenticationData.cpp index 75082041161..7a1091d8a1a 100644 --- a/src/Parsers/Access/ASTAuthenticationData.cpp +++ b/src/Parsers/Access/ASTAuthenticationData.cpp @@ -216,7 +216,8 @@ bool ASTAuthenticationData::hasSecretParts() const auto auth_type = *type; if ((auth_type == AuthenticationType::PLAINTEXT_PASSWORD) || (auth_type == AuthenticationType::SHA256_PASSWORD) - || (auth_type == AuthenticationType::DOUBLE_SHA1_PASSWORD)) + || (auth_type == AuthenticationType::DOUBLE_SHA1_PASSWORD) + || (auth_type == AuthenticationType::BCRYPT_PASSWORD)) return true; return childrenHaveSecretParts(); diff --git a/tests/queries/0_stateless/01292_create_user.reference b/tests/queries/0_stateless/01292_create_user.reference index 3df69ae2669..ed894382cc9 100644 --- a/tests/queries/0_stateless/01292_create_user.reference +++ b/tests/queries/0_stateless/01292_create_user.reference @@ -20,6 +20,7 @@ CREATE USER u2_01292 IDENTIFIED WITH sha256_password CREATE USER u3_01292 IDENTIFIED WITH sha256_password CREATE USER u4_01292 IDENTIFIED WITH plaintext_password CREATE USER u5_01292 IDENTIFIED WITH no_password +-- no passwords or hashes in query_log -- host CREATE USER u1_01292 IDENTIFIED WITH no_password CREATE USER u2_01292 IDENTIFIED WITH no_password HOST NONE diff --git a/tests/queries/0_stateless/01292_create_user.sql b/tests/queries/0_stateless/01292_create_user.sql index 41633b3d423..972aca6a57e 100644 --- a/tests/queries/0_stateless/01292_create_user.sql +++ b/tests/queries/0_stateless/01292_create_user.sql @@ -52,6 +52,17 @@ SHOW CREATE USER u2_01292; SHOW CREATE USER u3_01292; SHOW CREATE USER u4_01292; SHOW CREATE USER u5_01292; + +SELECT '-- no passwords or hashes in query_log'; +SELECT query +FROM system.query_log +WHERE + query NOT LIKE '%query_log%' AND + (query LIKE '%qwe123%' OR + query LIKE '%18138372FAD4B94533CD4881F03DC6C69296DD897234E0CEE83F727E2E6B1F63%' OR + query LIKE '%8DCDD69CE7D121DE8013062AEAEB2A148910D50E%' OR + query like '%$2a$12$rz5iy2LhuwBezsM88ZzWiemOVUeJ94xHTzwAlLMDhTzwUxOHaY64q%'); + DROP USER u1_01292, u2_01292, u3_01292, u4_01292, u5_01292, u6_01292, u7_01292, u8_01292, u9_01292; SELECT '-- host'; From cf8f2f629339ad84ae659b257a76d8bb99f84ed6 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 30 Sep 2024 20:18:01 +0000 Subject: [PATCH 20/65] Fix Dynamic type creation of JSON subcolumn --- src/DataTypes/DataTypeObject.cpp | 2 +- .../03246_json_subcolumn_correct_type.reference | 5 +++++ .../0_stateless/03246_json_subcolumn_correct_type.sql | 7 +++++++ 3 files changed, 13 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03246_json_subcolumn_correct_type.reference create mode 100644 tests/queries/0_stateless/03246_json_subcolumn_correct_type.sql diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 3a834c700df..8bb3c664249 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -404,7 +404,7 @@ std::unique_ptr DataTypeObject::getDynamicSubcolu else { res = std::make_unique(std::make_shared()); - res->type = std::make_shared(); + res->type = std::make_shared(max_dynamic_types); } /// If column was provided, we should create a column for requested subcolumn. diff --git a/tests/queries/0_stateless/03246_json_subcolumn_correct_type.reference b/tests/queries/0_stateless/03246_json_subcolumn_correct_type.reference new file mode 100644 index 00000000000..493e911e75e --- /dev/null +++ b/tests/queries/0_stateless/03246_json_subcolumn_correct_type.reference @@ -0,0 +1,5 @@ +Dynamic(max_types=1) +Dynamic(max_types=1) +1 +1 +1 diff --git a/tests/queries/0_stateless/03246_json_subcolumn_correct_type.sql b/tests/queries/0_stateless/03246_json_subcolumn_correct_type.sql new file mode 100644 index 00000000000..c21228ec33f --- /dev/null +++ b/tests/queries/0_stateless/03246_json_subcolumn_correct_type.sql @@ -0,0 +1,7 @@ +set allow_experimental_json_type=1; +drop table if exists test; +create table test (json JSON(max_dynamic_types=1)) engine=Memory; +insert into test values ('{"c0" : 1}'), ('{"c0" : 2}'); +select toTypeName(json.c0) from test; +SELECT 1 FROM (SELECT 1 AS c0) tx FULL OUTER JOIN test ON test.json.Float32 = tx.c0; +drop table test; From c64cc40ba5f5b38427d6ac327b361f0237c62790 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 30 Sep 2024 20:25:55 +0000 Subject: [PATCH 21/65] Better test --- tests/queries/0_stateless/01292_create_user.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01292_create_user.sql b/tests/queries/0_stateless/01292_create_user.sql index 972aca6a57e..756f4964e7d 100644 --- a/tests/queries/0_stateless/01292_create_user.sql +++ b/tests/queries/0_stateless/01292_create_user.sql @@ -54,11 +54,12 @@ SHOW CREATE USER u4_01292; SHOW CREATE USER u5_01292; SELECT '-- no passwords or hashes in query_log'; +SYSTEM FLUSH LOGS; SELECT query FROM system.query_log WHERE query NOT LIKE '%query_log%' AND - (query LIKE '%qwe123%' OR + (query LIKE '%qwe123%' OR query LIKE '%123qwe%' OR query LIKE '%18138372FAD4B94533CD4881F03DC6C69296DD897234E0CEE83F727E2E6B1F63%' OR query LIKE '%8DCDD69CE7D121DE8013062AEAEB2A148910D50E%' OR query like '%$2a$12$rz5iy2LhuwBezsM88ZzWiemOVUeJ94xHTzwAlLMDhTzwUxOHaY64q%'); From de3e3c877b8361fa066bb2c343ad95b0901384ac Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 30 Sep 2024 20:41:55 +0000 Subject: [PATCH 22/65] Fix test --- tests/queries/0_stateless/01292_create_user.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01292_create_user.sql b/tests/queries/0_stateless/01292_create_user.sql index 756f4964e7d..d320b2b9713 100644 --- a/tests/queries/0_stateless/01292_create_user.sql +++ b/tests/queries/0_stateless/01292_create_user.sql @@ -58,7 +58,7 @@ SYSTEM FLUSH LOGS; SELECT query FROM system.query_log WHERE - query NOT LIKE '%query_log%' AND + query NOT LIKE '%query_log%' AND event_date >= yesterday() AND current_database = currentDatabase() AND (query LIKE '%qwe123%' OR query LIKE '%123qwe%' OR query LIKE '%18138372FAD4B94533CD4881F03DC6C69296DD897234E0CEE83F727E2E6B1F63%' OR query LIKE '%8DCDD69CE7D121DE8013062AEAEB2A148910D50E%' OR From 4452f354a5432fc1ed346c3a8fc95617cea2a0f3 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 30 Sep 2024 19:11:04 +0000 Subject: [PATCH 23/65] Silence UBSAN for integer overflows in some datetime functions --- src/Functions/DateTimeTransforms.h | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/src/Functions/DateTimeTransforms.h b/src/Functions/DateTimeTransforms.h index d6d533f16ed..f7075e72b5e 100644 --- a/src/Functions/DateTimeTransforms.h +++ b/src/Functions/DateTimeTransforms.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -178,7 +179,7 @@ struct ToStartOfDayImpl } static Int64 executeExtendedResult(Int32 d, const DateLUTImpl & time_zone) { - return time_zone.fromDayNum(ExtendedDayNum(d)) * DecimalUtils::scaleMultiplier(DataTypeDateTime64::default_scale); + return common::mulIgnoreOverflow(time_zone.fromDayNum(ExtendedDayNum(d)), DecimalUtils::scaleMultiplier(DataTypeDateTime64::default_scale)); } using FactorTransform = ZeroTransform; @@ -1980,22 +1981,19 @@ struct ToRelativeSubsecondNumImpl return t.value; if (scale > scale_multiplier) return t.value / (scale / scale_multiplier); - return static_cast(t.value) * static_cast((scale_multiplier / scale)); - /// Casting ^^: All integers are Int64, yet if t.value is big enough the multiplication can still - /// overflow which is UB. This place is too low-level and generic to check if t.value is sane. - /// Therefore just let it overflow safely and don't bother further. + return common::mulIgnoreOverflow(t.value, scale_multiplier / scale); } static Int64 execute(UInt32 t, const DateLUTImpl &) { - return t * scale_multiplier; + return common::mulIgnoreOverflow(static_cast(t), scale_multiplier); } static Int64 execute(Int32 d, const DateLUTImpl & time_zone) { - return static_cast(time_zone.fromDayNum(ExtendedDayNum(d))) * scale_multiplier; + return common::mulIgnoreOverflow(static_cast(time_zone.fromDayNum(ExtendedDayNum(d))), scale_multiplier); } static Int64 execute(UInt16 d, const DateLUTImpl & time_zone) { - return static_cast(time_zone.fromDayNum(DayNum(d)) * scale_multiplier); + return common::mulIgnoreOverflow(static_cast(time_zone.fromDayNum(DayNum(d))), scale_multiplier); } using FactorTransform = ZeroTransform; From f84e1e4d27a83cff8597dbd9459f14d8f2a8dd13 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 30 Sep 2024 21:44:29 +0000 Subject: [PATCH 24/65] Even better test --- .../0_stateless/01292_create_user.reference | 2 +- .../queries/0_stateless/01292_create_user.sql | 25 +++++++++---------- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/01292_create_user.reference b/tests/queries/0_stateless/01292_create_user.reference index ed894382cc9..186e4c8d75b 100644 --- a/tests/queries/0_stateless/01292_create_user.reference +++ b/tests/queries/0_stateless/01292_create_user.reference @@ -20,7 +20,6 @@ CREATE USER u2_01292 IDENTIFIED WITH sha256_password CREATE USER u3_01292 IDENTIFIED WITH sha256_password CREATE USER u4_01292 IDENTIFIED WITH plaintext_password CREATE USER u5_01292 IDENTIFIED WITH no_password --- no passwords or hashes in query_log -- host CREATE USER u1_01292 IDENTIFIED WITH no_password CREATE USER u2_01292 IDENTIFIED WITH no_password HOST NONE @@ -120,3 +119,4 @@ u4_01292 local_directory ['double_sha1_password'] ['{}'] ['::/0'] [] [] [] 1 [] \N u4_01292 \N 2 readonly 1 \N \N \N \N -- multiple authentication methods u1_01292 ['plaintext_password','kerberos','bcrypt_password','ldap'] ['{}','{"realm":"qwerty10"}','{}','{"server":"abc"}'] +-- no passwords or hashes in query_log diff --git a/tests/queries/0_stateless/01292_create_user.sql b/tests/queries/0_stateless/01292_create_user.sql index d320b2b9713..92eb139d06f 100644 --- a/tests/queries/0_stateless/01292_create_user.sql +++ b/tests/queries/0_stateless/01292_create_user.sql @@ -52,18 +52,6 @@ SHOW CREATE USER u2_01292; SHOW CREATE USER u3_01292; SHOW CREATE USER u4_01292; SHOW CREATE USER u5_01292; - -SELECT '-- no passwords or hashes in query_log'; -SYSTEM FLUSH LOGS; -SELECT query -FROM system.query_log -WHERE - query NOT LIKE '%query_log%' AND event_date >= yesterday() AND current_database = currentDatabase() AND - (query LIKE '%qwe123%' OR query LIKE '%123qwe%' OR - query LIKE '%18138372FAD4B94533CD4881F03DC6C69296DD897234E0CEE83F727E2E6B1F63%' OR - query LIKE '%8DCDD69CE7D121DE8013062AEAEB2A148910D50E%' OR - query like '%$2a$12$rz5iy2LhuwBezsM88ZzWiemOVUeJ94xHTzwAlLMDhTzwUxOHaY64q%'); - DROP USER u1_01292, u2_01292, u3_01292, u4_01292, u5_01292, u6_01292, u7_01292, u8_01292, u9_01292; SELECT '-- host'; @@ -247,6 +235,17 @@ DROP USER u1_01292, u2_01292, u3_01292, u4_01292, u5_01292; DROP ROLE r1_01292, r2_01292; SELECT '-- multiple authentication methods'; -CREATE USER u1_01292 IDENTIFIED WITH plaintext_password by '1', kerberos REALM 'qwerty10', bcrypt_password by '3', ldap SERVER 'abc'; +CREATE USER u1_01292 IDENTIFIED WITH plaintext_password by 'qwe123', kerberos REALM 'qwerty10', bcrypt_password by '123qwe', ldap SERVER 'abc'; SELECT name, auth_type, auth_params FROM system.users WHERE name = 'u1_01292' ORDER BY name; DROP USER u1_01292; + +SELECT '-- no passwords or hashes in query_log'; +SYSTEM FLUSH LOGS; +SELECT query +FROM system.query_log +WHERE + query NOT LIKE '%query_log%' AND event_date >= yesterday() AND current_database = currentDatabase() AND + (query LIKE '%qwe123%' OR query LIKE '%123qwe%' OR + query LIKE '%18138372FAD4B94533CD4881F03DC6C69296DD897234E0CEE83F727E2E6B1F63%' OR + query LIKE '%8DCDD69CE7D121DE8013062AEAEB2A148910D50E%' OR + query like '%$2a$12$rz5iy2LhuwBezsM88ZzWiemOVUeJ94xHTzwAlLMDhTzwUxOHaY64q%'); From 32b6b159b0fad805b770056eada296b13b4e5cfd Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 30 Sep 2024 21:48:17 +0000 Subject: [PATCH 25/65] Fix 'loop' crashing when reading from empty MergeTree table --- src/Processors/QueryPlan/ReadFromLoopStep.cpp | 19 +++++++++++-------- .../0_stateless/03147_table_function_loop.sql | 3 +++ 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromLoopStep.cpp b/src/Processors/QueryPlan/ReadFromLoopStep.cpp index 2e5fa3ec9f7..4c3ad241853 100644 --- a/src/Processors/QueryPlan/ReadFromLoopStep.cpp +++ b/src/Processors/QueryPlan/ReadFromLoopStep.cpp @@ -63,17 +63,20 @@ namespace DB processed_stage, max_block_size, num_streams); - auto builder = plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(context), - BuildQueryPipelineSettings::fromContext(context)); - QueryPlanResourceHolder resources; - auto pipe = QueryPipelineBuilder::getPipe(std::move(*builder), resources); - query_pipeline = QueryPipeline(std::move(pipe)); - executor = std::make_unique(query_pipeline); + if (plan.isInitialized()) + { + auto builder = plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(context), + BuildQueryPipelineSettings::fromContext(context)); + QueryPlanResourceHolder resources; + auto pipe = QueryPipelineBuilder::getPipe(std::move(*builder), resources); + query_pipeline = QueryPipeline(std::move(pipe)); + executor = std::make_unique(query_pipeline); + } loop = true; } Chunk chunk; - if (executor->pull(chunk)) + if (executor && executor->pull(chunk)) { if (chunk) { diff --git a/tests/queries/0_stateless/03147_table_function_loop.sql b/tests/queries/0_stateless/03147_table_function_loop.sql index e10155fe8b3..797fb44cad3 100644 --- a/tests/queries/0_stateless/03147_table_function_loop.sql +++ b/tests/queries/0_stateless/03147_table_function_loop.sql @@ -4,6 +4,9 @@ SELECT * FROM loop(numbers(3)) LIMIT 10; SELECT * FROM loop (numbers(3)) LIMIT 10 settings max_block_size = 1; CREATE TABLE t (n Int8) ENGINE=MergeTree ORDER BY n; + +SELECT * FROM loop(t) LIMIT 15; -- { serverError TOO_MANY_RETRIES_TO_FETCH_PARTS } + INSERT INTO t SELECT * FROM numbers(10); SELECT * FROM loop({CLICKHOUSE_DATABASE:Identifier}.t) LIMIT 15; From 15d69d696e4533c13df3d13444f184b9055c4f6d Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Mon, 30 Sep 2024 22:06:19 +0000 Subject: [PATCH 26/65] Test --- tests/queries/0_stateless/02477_age_date32.reference | 3 +++ tests/queries/0_stateless/02477_age_date32.sql | 2 ++ 2 files changed, 5 insertions(+) diff --git a/tests/queries/0_stateless/02477_age_date32.reference b/tests/queries/0_stateless/02477_age_date32.reference index 69f27a10acc..02505439b12 100644 --- a/tests/queries/0_stateless/02477_age_date32.reference +++ b/tests/queries/0_stateless/02477_age_date32.reference @@ -167,3 +167,6 @@ SELECT age('day', materialize(toDate32('2015-08-18', 'UTC')), materialize(toDate 1 SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDate32('2015-08-19', 'UTC')), 'UTC'); 1 +SELECT age('minute', toDate32(1234567890123456, 'UTC'), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'), 'UTC') + +-196176960 diff --git a/tests/queries/0_stateless/02477_age_date32.sql b/tests/queries/0_stateless/02477_age_date32.sql index 43ff458c2d1..302c8870820 100644 --- a/tests/queries/0_stateless/02477_age_date32.sql +++ b/tests/queries/0_stateless/02477_age_date32.sql @@ -99,3 +99,5 @@ SELECT age('day', materialize(toDate32('2015-08-18', 'UTC')), materialize(toDate SELECT age('day', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), materialize(toDate32('2015-08-19', 'UTC')), 'UTC'); SELECT age('day', materialize(toDate32('2015-08-18', 'UTC')), materialize(toDate('2015-08-19', 'UTC')), 'UTC'); SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDate32('2015-08-19', 'UTC')), 'UTC'); + +SELECT age('minute', toDate32(1234567890123456, 'UTC'), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'), 'UTC') From 4bfad1ccd69aea89e46f8b5012fde6c86b324707 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 30 Sep 2024 20:28:06 +0200 Subject: [PATCH 27/65] Improve reqgenerator a little bit --- docker/reqgenerator.py | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/docker/reqgenerator.py b/docker/reqgenerator.py index cbd25718dee..a9b8dd4d0b2 100644 --- a/docker/reqgenerator.py +++ b/docker/reqgenerator.py @@ -7,14 +7,20 @@ import subprocess import sys -def build_docker_deps(image_name, imagedir): - cmd = f"""docker run --entrypoint "/bin/bash" {image_name} -c "pip install pipdeptree 2>/dev/null 1>/dev/null && pipdeptree --freeze --warn silence | sed 's/ \+//g' | sort | uniq" > {imagedir}/requirements.txt""" +def build_docker_deps(image_name: str, imagedir: str) -> None: + print("Fetch the newest manifest for", image_name) + pip_cmd = ( + "pip install pipdeptree 2>/dev/null 1>/dev/null && pipdeptree --freeze " + "--warn silence --exclude pipdeptree" + ) + cmd = rf"""docker run --rm --entrypoint "/bin/bash" {image_name} -c "{pip_cmd} | sed 's/ \+//g' | sort | uniq" > {imagedir}/requirements.txt""" + print("Running the command:", cmd) subprocess.check_call(cmd, shell=True) def check_docker_file_install_with_pip(filepath): image_name = None - with open(filepath, "r") as f: + with open(filepath, "r", encoding="utf-8") as f: for line in f: if "docker build" in line: arr = line.split(" ") @@ -25,7 +31,7 @@ def check_docker_file_install_with_pip(filepath): return image_name, False -def process_affected_images(images_dir): +def process_affected_images(images_dir: str) -> None: for root, _dirs, files in os.walk(images_dir): for f in files: if f == "Dockerfile": From 985bd796d32708abe09ec1d130775a54f8285a1f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 30 Sep 2024 22:13:56 +0200 Subject: [PATCH 28/65] Simplify retry decorator, use it in tests --- tests/integration/helpers/cluster.py | 6 +-- tests/integration/helpers/retry_decorator.py | 41 ++++++++----------- .../test_keeper_s3_snapshot/test.py | 20 +++++++-- 3 files changed, 35 insertions(+), 32 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f5f87947c0f..43b5d291cf4 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2837,7 +2837,7 @@ class ClickHouseCluster: "Got exception pulling images: %s", kwargs["exception"] ) - retry(log_function=logging_pulling_images)(run_and_check)(images_pull_cmd) + retry(log_function=logging_pulling_images)(run_and_check, images_pull_cmd) if self.with_zookeeper_secure and self.base_zookeeper_cmd: logging.debug("Setup ZooKeeper Secure") @@ -3118,9 +3118,7 @@ class ClickHouseCluster: retry( log_function=logging_azurite_initialization, - )( - run_and_check - )(azurite_start_cmd) + )(run_and_check, azurite_start_cmd) self.up_called = True logging.info("Trying to connect to Azurite") self.wait_azurite_to_start() diff --git a/tests/integration/helpers/retry_decorator.py b/tests/integration/helpers/retry_decorator.py index 771fab49d61..5f2e600edab 100644 --- a/tests/integration/helpers/retry_decorator.py +++ b/tests/integration/helpers/retry_decorator.py @@ -11,29 +11,22 @@ def retry( log_function=None, # should take **kwargs or arguments: `retry_number`, `exception` and `sleep_time` retriable_expections_list: List[Type[BaseException]] = [Exception], ): - def inner(func): - def wrapper(*args, **kwargs): - current_delay = delay - for retry in range(retries): - try: - func(*args, **kwargs) - break - except Exception as e: - should_retry = False - for retriable_exception in retriable_expections_list: - if isinstance(e, retriable_exception): - should_retry = True - break - if not should_retry or (retry == retries - 1): - raise e - sleep_time = current_delay + random.uniform(0, jitter) - if log_function is not None: - log_function( - retry_number=retry, exception=e, sleep_time=sleep_time - ) - time.sleep(sleep_time) - current_delay *= backoff - - return wrapper + def inner(func, *args, **kwargs): + current_delay = delay + for retry in range(retries): + try: + func(*args, **kwargs) + break + except Exception as e: + should_retry = (retry <= retries - 1) and any( + isinstance(e, re) for re in retriable_expections_list + ) + if not should_retry: + raise e + sleep_time = current_delay + random.uniform(0, jitter) + if log_function is not None: + log_function(retry_number=retry, exception=e, sleep_time=sleep_time) + time.sleep(sleep_time) + current_delay *= backoff return inner diff --git a/tests/integration/test_keeper_s3_snapshot/test.py b/tests/integration/test_keeper_s3_snapshot/test.py index a7f99bf5bee..3a7a38946b7 100644 --- a/tests/integration/test_keeper_s3_snapshot/test.py +++ b/tests/integration/test_keeper_s3_snapshot/test.py @@ -4,10 +4,10 @@ from time import sleep import pytest from kazoo.client import KazooClient from minio.deleteobjects import DeleteObject -from retry import retry -import helpers.keeper_utils as keeper_utils +from helpers import keeper_utils from helpers.cluster import ClickHouseCluster +from helpers.retry_decorator import retry # from kazoo.protocol.serialization import Connect, read_buffer, write_buffer @@ -110,7 +110,13 @@ def test_s3_upload(started_cluster): cluster.minio_client.remove_object("snapshots", s.object_name) # Keeper sends snapshots asynchornously, hence we need to retry. - @retry(AssertionError, tries=10, delay=2) + @retry( + retries=10, + delay=2, + jitter=0, + backoff=0, + retriable_expections_list=[AssertionError], + ) def _check_snapshots(): assert set(get_saved_snapshots()) == set( [ @@ -133,7 +139,13 @@ def test_s3_upload(started_cluster): for _ in range(200): node2_zk.create("/test", sequence=True) - @retry(AssertionError, tries=10, delay=2) + @retry( + retries=10, + delay=2, + jitter=0, + backoff=0, + retriable_expections_list=[AssertionError], + ) def _check_snapshots_without_quorum(): assert len(get_saved_snapshots()) > 4 From 650b7420edd0830ad63c9507bacda1dd58d81af2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 30 Sep 2024 22:42:47 +0200 Subject: [PATCH 29/65] Improve the @retry API --- tests/integration/helpers/retry_decorator.py | 8 +++++--- .../test_keeper_s3_snapshot/test.py | 20 +++++-------------- 2 files changed, 10 insertions(+), 18 deletions(-) diff --git a/tests/integration/helpers/retry_decorator.py b/tests/integration/helpers/retry_decorator.py index 5f2e600edab..17eef42717a 100644 --- a/tests/integration/helpers/retry_decorator.py +++ b/tests/integration/helpers/retry_decorator.py @@ -4,13 +4,15 @@ from typing import List, Type def retry( + *exceptions: Type[BaseException], retries: int = 5, delay: float = 1, backoff: float = 1.5, jitter: float = 2, log_function=None, # should take **kwargs or arguments: `retry_number`, `exception` and `sleep_time` - retriable_expections_list: List[Type[BaseException]] = [Exception], ): + exceptions = exceptions or (Exception,) + def inner(func, *args, **kwargs): current_delay = delay for retry in range(retries): @@ -18,8 +20,8 @@ def retry( func(*args, **kwargs) break except Exception as e: - should_retry = (retry <= retries - 1) and any( - isinstance(e, re) for re in retriable_expections_list + should_retry = (retry < retries - 1) and any( + isinstance(e, re) for re in exceptions ) if not should_retry: raise e diff --git a/tests/integration/test_keeper_s3_snapshot/test.py b/tests/integration/test_keeper_s3_snapshot/test.py index 3a7a38946b7..54db41d9e0d 100644 --- a/tests/integration/test_keeper_s3_snapshot/test.py +++ b/tests/integration/test_keeper_s3_snapshot/test.py @@ -110,13 +110,6 @@ def test_s3_upload(started_cluster): cluster.minio_client.remove_object("snapshots", s.object_name) # Keeper sends snapshots asynchornously, hence we need to retry. - @retry( - retries=10, - delay=2, - jitter=0, - backoff=0, - retriable_expections_list=[AssertionError], - ) def _check_snapshots(): assert set(get_saved_snapshots()) == set( [ @@ -127,7 +120,7 @@ def test_s3_upload(started_cluster): ] ) - _check_snapshots() + retry(AssertionError, retries=10, delay=2, jitter=0, backoff=1)(_check_snapshots) destroy_zk_client(node1_zk) node1.stop_clickhouse(kill=True) @@ -139,16 +132,13 @@ def test_s3_upload(started_cluster): for _ in range(200): node2_zk.create("/test", sequence=True) - @retry( - retries=10, - delay=2, - jitter=0, - backoff=0, - retriable_expections_list=[AssertionError], - ) def _check_snapshots_without_quorum(): assert len(get_saved_snapshots()) > 4 + retry(AssertionError, retries=10, delay=2, jitter=0, backoff=1)( + _check_snapshots_without_quorum + ) + _check_snapshots_without_quorum() success_upload_message = "Successfully uploaded" From 2e799b00f8fb4858aabf75e80f5621a598097cfa Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 30 Sep 2024 22:48:11 +0200 Subject: [PATCH 30/65] Ignore dependencies without installed version --- docker/reqgenerator.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/reqgenerator.py b/docker/reqgenerator.py index a9b8dd4d0b2..f7846f3e842 100644 --- a/docker/reqgenerator.py +++ b/docker/reqgenerator.py @@ -13,7 +13,7 @@ def build_docker_deps(image_name: str, imagedir: str) -> None: "pip install pipdeptree 2>/dev/null 1>/dev/null && pipdeptree --freeze " "--warn silence --exclude pipdeptree" ) - cmd = rf"""docker run --rm --entrypoint "/bin/bash" {image_name} -c "{pip_cmd} | sed 's/ \+//g' | sort | uniq" > {imagedir}/requirements.txt""" + cmd = rf"""docker run --rm --entrypoint "/bin/bash" {image_name} -c "{pip_cmd} | sed '/=/!d;s/\s//g' | sort -u" > {imagedir}/requirements.txt""" print("Running the command:", cmd) subprocess.check_call(cmd, shell=True) From c7e478e2341904d00b94c496c8090911ae40fdf0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 30 Sep 2024 22:49:10 +0200 Subject: [PATCH 31/65] Make dependabot happy --- .../test/integration/runner/requirements.txt | 21 ++++++++----------- 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index 2c9df73ccca..1f472cf4186 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -1,6 +1,6 @@ PyHDFS==0.3.1 -PyJWT==2.3.0 -PyMySQL==1.1.0 +PyJWT==2.4.0 +PyMySQL==1.1.1 PyNaCl==1.5.0 PyYAML==5.3.1 SecretStorage==3.3.1 @@ -9,7 +9,7 @@ argon2-cffi==23.1.0 async-timeout==4.0.3 asyncio==3.4.3 attrs==23.2.0 -avro==1.10.2 +avro==1.11.3 azure-core==1.30.1 azure-storage-blob==12.19.0 bcrypt==4.1.3 @@ -24,10 +24,10 @@ cffi==1.16.0 charset-normalizer==3.3.2 click==8.1.7 confluent-kafka==2.3.0 -cryptography==3.4.8 +cryptography==42.0.0 dbus-python==1.2.18 -decorator==5.1.1 delta-spark==2.3.0 +deltalake==0.16.0 dict2xml==1.7.4 dicttoxml==1.7.16 distro-info==1.1+ubuntu0.2 @@ -63,17 +63,17 @@ lz4==4.3.3 minio==7.2.3 more-itertools==8.10.0 nats-py==2.6.0 +numpy==2.1.0 oauthlib==3.2.0 packaging==24.0 paramiko==3.4.0 pika==1.2.0 pip==24.1.1 -pipdeptree==2.23.0 pluggy==1.5.0 protobuf==4.25.2 psycopg2-binary==2.9.6 py4j==0.10.9.5 -py==1.11.0 +pyarrow-hotfix==0.6 pyarrow==17.0.0 pycparser==2.22 pycryptodome==3.20.0 @@ -96,9 +96,8 @@ pytz==2023.3.post1 redis==5.0.1 requests-kerberos==0.14.0 requests==2.31.0 -retry==0.9.2 s3transfer==0.10.1 -setuptools==59.6.0 +setuptools==70.0.0 simplejson==3.19.2 six==1.16.0 soupsieve==2.5 @@ -110,7 +109,5 @@ unattended-upgrades==0.1 urllib3==2.0.7 wadllib==1.3.6 websocket-client==0.59.0 -wheel==0.37.1 +wheel==0.38.1 zipp==1.0.0 -deltalake==0.16.0 - From 0308de23c06788cdd300c1b450554345036664e5 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 30 Sep 2024 23:20:09 +0200 Subject: [PATCH 32/65] Migrate to GO version of docker-compose --- docker/test/integration/runner/Dockerfile | 2 +- docker/test/integration/runner/requirements.txt | 13 ++----------- 2 files changed, 3 insertions(+), 12 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index d62009f1be3..403409072f4 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -48,7 +48,7 @@ RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - \ && add-apt-repository "deb https://download.docker.com/linux/ubuntu $(lsb_release -c -s) ${DOCKER_CHANNEL}" \ && apt-get update \ && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ - docker-ce='5:23.*' \ + docker-ce='5:23.*' docker-compose-plugin='2.29.*' \ && rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index 1f472cf4186..8e1d596ea15 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -1,14 +1,13 @@ +PyGObject==3.42.1 PyHDFS==0.3.1 PyJWT==2.4.0 PyMySQL==1.1.1 PyNaCl==1.5.0 -PyYAML==5.3.1 SecretStorage==3.3.1 argon2-cffi-bindings==21.2.0 argon2-cffi==23.1.0 async-timeout==4.0.3 asyncio==3.4.3 -attrs==23.2.0 avro==1.11.3 azure-core==1.30.1 azure-storage-blob==12.19.0 @@ -31,11 +30,7 @@ deltalake==0.16.0 dict2xml==1.7.4 dicttoxml==1.7.16 distro-info==1.1+ubuntu0.2 -distro==1.7.0 -docker-compose==1.29.2 docker==6.1.3 -dockerpty==0.4.1 -docopt==0.6.2 exceptiongroup==1.2.1 execnet==2.1.1 geomet==0.2.1.post1 @@ -49,7 +44,6 @@ iniconfig==2.0.0 isodate==0.6.1 jeepney==0.7.1 jmespath==1.0.1 -jsonschema==3.2.0 jwcrypto==1.5.6 kafka-python==2.0.2 kazoo==2.9.0 @@ -79,7 +73,6 @@ pycparser==2.22 pycryptodome==3.20.0 pymongo==3.11.0 pyparsing==2.4.7 -pyrsistent==0.20.0 pyspark==3.3.2 pyspnego==0.10.2 pytest-order==1.0.0 @@ -91,7 +84,6 @@ pytest-xdist==3.5.0 pytest==7.4.4 python-apt==2.4.0+ubuntu3 python-dateutil==2.9.0.post0 -python-dotenv==0.21.1 pytz==2023.3.post1 redis==5.0.1 requests-kerberos==0.14.0 @@ -101,13 +93,12 @@ setuptools==70.0.0 simplejson==3.19.2 six==1.16.0 soupsieve==2.5 -texttable==1.7.0 tomli==2.0.1 typing_extensions==4.11.0 tzlocal==2.1 unattended-upgrades==0.1 urllib3==2.0.7 wadllib==1.3.6 -websocket-client==0.59.0 +websocket-client==1.8.0 wheel==0.38.1 zipp==1.0.0 From ce4ddf73364d5e6877917f26a93890195bbe89e0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 30 Sep 2024 23:43:36 +0200 Subject: [PATCH 33/65] Ignore system packages in pipdeptree --- docker/reqgenerator.py | 6 +++++- docker/test/integration/runner/requirements.txt | 2 -- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/docker/reqgenerator.py b/docker/reqgenerator.py index f7846f3e842..3d6ab2136db 100644 --- a/docker/reqgenerator.py +++ b/docker/reqgenerator.py @@ -13,7 +13,11 @@ def build_docker_deps(image_name: str, imagedir: str) -> None: "pip install pipdeptree 2>/dev/null 1>/dev/null && pipdeptree --freeze " "--warn silence --exclude pipdeptree" ) - cmd = rf"""docker run --rm --entrypoint "/bin/bash" {image_name} -c "{pip_cmd} | sed '/=/!d;s/\s//g' | sort -u" > {imagedir}/requirements.txt""" + # /=/!d - remove dependencies without pin + # ubuntu - ignore system packages + # \s - remove spaces + sed = r"sed '/==/!d; /==.*+ubuntu/d; s/\s//g'" + cmd = rf"""docker run --rm --entrypoint "/bin/bash" {image_name} -c "{pip_cmd} | {sed} | sort -u" > {imagedir}/requirements.txt""" print("Running the command:", cmd) subprocess.check_call(cmd, shell=True) diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index 8e1d596ea15..7ece526a580 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -29,7 +29,6 @@ delta-spark==2.3.0 deltalake==0.16.0 dict2xml==1.7.4 dicttoxml==1.7.16 -distro-info==1.1+ubuntu0.2 docker==6.1.3 exceptiongroup==1.2.1 execnet==2.1.1 @@ -82,7 +81,6 @@ pytest-reportlog==0.4.0 pytest-timeout==2.2.0 pytest-xdist==3.5.0 pytest==7.4.4 -python-apt==2.4.0+ubuntu3 python-dateutil==2.9.0.post0 pytz==2023.3.post1 redis==5.0.1 From 171ffa80b9986d4633a81578e11781c514831e5d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 30 Sep 2024 23:56:06 +0200 Subject: [PATCH 34/65] Add pycairo pin + build dependency --- docker/test/integration/runner/requirements.txt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index 7ece526a580..97e69a1f5c8 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -68,6 +68,7 @@ psycopg2-binary==2.9.6 py4j==0.10.9.5 pyarrow-hotfix==0.6 pyarrow==17.0.0 +pycairo==1.27.0 pycparser==2.22 pycryptodome==3.20.0 pymongo==3.11.0 @@ -81,6 +82,7 @@ pytest-reportlog==0.4.0 pytest-timeout==2.2.0 pytest-xdist==3.5.0 pytest==7.4.4 +python-meson==0.16.0 python-dateutil==2.9.0.post0 pytz==2023.3.post1 redis==5.0.1 From bfd3543bf83cc4ddb31fd7957a70f0f608c4ae21 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Oct 2024 00:10:09 +0200 Subject: [PATCH 35/65] Try to remove pygobject --- docker/test/integration/runner/requirements.txt | 3 --- 1 file changed, 3 deletions(-) diff --git a/docker/test/integration/runner/requirements.txt b/docker/test/integration/runner/requirements.txt index 97e69a1f5c8..4802623abd6 100644 --- a/docker/test/integration/runner/requirements.txt +++ b/docker/test/integration/runner/requirements.txt @@ -1,4 +1,3 @@ -PyGObject==3.42.1 PyHDFS==0.3.1 PyJWT==2.4.0 PyMySQL==1.1.1 @@ -68,7 +67,6 @@ psycopg2-binary==2.9.6 py4j==0.10.9.5 pyarrow-hotfix==0.6 pyarrow==17.0.0 -pycairo==1.27.0 pycparser==2.22 pycryptodome==3.20.0 pymongo==3.11.0 @@ -82,7 +80,6 @@ pytest-reportlog==0.4.0 pytest-timeout==2.2.0 pytest-xdist==3.5.0 pytest==7.4.4 -python-meson==0.16.0 python-dateutil==2.9.0.post0 pytz==2023.3.post1 redis==5.0.1 From f76e6ea24bbe34623d9969e6e35264a1b88b515c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Oct 2024 08:03:22 +0200 Subject: [PATCH 36/65] Adjust integration tests to docker compose v2 --- tests/integration/README.md | 5 +- .../compose/docker_compose_clickhouse.yml | 2 +- tests/integration/conftest.py | 2 +- tests/integration/helpers/cluster.py | 102 +++++++++++------- tests/integration/runner | 2 +- tests/integration/test_dotnet_client/test.py | 3 +- tests/integration/test_mysql_protocol/test.py | 12 ++- 7 files changed, 80 insertions(+), 48 deletions(-) diff --git a/tests/integration/README.md b/tests/integration/README.md index a149a4d60ef..b246eeb0674 100644 --- a/tests/integration/README.md +++ b/tests/integration/README.md @@ -14,7 +14,7 @@ Don't use Docker from your system repository. * [pip](https://pypi.python.org/pypi/pip) and `libpq-dev`. To install: `sudo apt-get install python3-pip libpq-dev zlib1g-dev libcrypto++-dev libssl-dev libkrb5-dev python3-dev` * [py.test](https://docs.pytest.org/) testing framework. To install: `sudo -H pip install pytest` -* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: +* [docker compose](https://docs.docker.com/compose/) and additional python libraries. To install: ```bash sudo -H pip install \ @@ -24,7 +24,6 @@ sudo -H pip install \ confluent-kafka \ dicttoxml \ docker \ - docker-compose \ grpcio \ grpcio-tools \ kafka-python \ @@ -48,7 +47,7 @@ sudo -H pip install \ nats-py ``` -(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python3-pytest python3-dicttoxml python3-docker python3-pymysql python3-protobuf python3-pymongo python3-tzlocal python3-kazoo python3-psycopg2 kafka-python python3-pytest-timeout python3-minio` +(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose-v2 python3-pytest python3-dicttoxml python3-docker python3-pymysql python3-protobuf python3-pymongo python3-tzlocal python3-kazoo python3-psycopg2 kafka-python python3-pytest-timeout python3-minio` Some tests have other dependencies, e.g. spark. See docker/test/integration/runner/Dockerfile for how to install those. See docker/test/integration/runner/dockerd-entrypoint.sh for environment variables that need to be set (e.g. JAVA_PATH). diff --git a/tests/integration/compose/docker_compose_clickhouse.yml b/tests/integration/compose/docker_compose_clickhouse.yml index fdd124ede91..9702649f1b8 100644 --- a/tests/integration/compose/docker_compose_clickhouse.yml +++ b/tests/integration/compose/docker_compose_clickhouse.yml @@ -1,5 +1,5 @@ version: '2.3' -# Used to pre-pull images with docker-compose +# Used to pre-pull images with docker compose services: clickhouse1: image: clickhouse/integration-test diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index 88ce8640c97..1e59ae14820 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -90,7 +90,7 @@ def cleanup_environment(): nothrow=True, ) logging.debug("Unstopped containers killed") - r = run_and_check(["docker-compose", "ps", "--services", "--all"]) + r = run_and_check(["docker", "compose", "ps", "--services", "--all"]) logging.debug("Docker ps before start:%s", r.stdout) else: logging.debug("No running containers") diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 43b5d291cf4..87571b7115d 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -399,10 +399,10 @@ def get_instances_dir(name): run_id = os.environ.get("INTEGRATION_TESTS_RUN_ID", "") if name: - instances_dir_name += "_" + name + instances_dir_name += "-" + name if run_id: - instances_dir_name += "_" + shlex.quote(run_id) + instances_dir_name += "-" + shlex.quote(run_id) return instances_dir_name @@ -487,8 +487,8 @@ class ClickHouseCluster: self.instances_dir_name = get_instances_dir(self.name) xdist_worker = os.getenv("PYTEST_XDIST_WORKER") if xdist_worker: - self.project_name += f"_{xdist_worker}" - self.instances_dir_name += f"_{xdist_worker}" + self.project_name += f"-{xdist_worker}" + self.instances_dir_name += f"-{xdist_worker}" self.instances_dir = p.join(self.base_dir, self.instances_dir_name) self.docker_logs_path = p.join(self.instances_dir, "docker.log") @@ -509,7 +509,7 @@ class ClickHouseCluster: self.docker_api_version = os.environ.get("DOCKER_API_VERSION") self.docker_base_tag = os.environ.get("DOCKER_BASE_TAG", "latest") - self.base_cmd = ["docker-compose"] + self.base_cmd = ["docker", "compose"] if custom_dockerd_host: self.base_cmd += ["--host", custom_dockerd_host] self.base_cmd += ["--env-file", self.env_file] @@ -971,10 +971,10 @@ class ClickHouseCluster: # Returns the list of currently running docker containers corresponding to this ClickHouseCluster. def get_running_containers(self): # docker-compose names containers using the following formula: - # container_name = project_name + '_' + instance_name + '_1' + # container_name = project_name + '-' + instance_name + '-1' # We need to have "^/" and "$" in the "--filter name" option below to filter by exact name of the container, see # https://stackoverflow.com/questions/48767760/how-to-make-docker-container-ls-f-name-filter-by-exact-name - filter_name = f"^/{self.project_name}_.*_1$" + filter_name = f"^/{self.project_name}-.*-1$" # We want the command "docker container list" to show only containers' ID and their names, separated by colon. format = "{{.ID}}:{{.Names}}" containers = run_and_check( @@ -1021,7 +1021,8 @@ class ClickHouseCluster: self.with_zookeeper_secure = True self.base_cmd.extend(["--file", zookeeper_docker_compose_path]) self.base_zookeeper_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1053,7 +1054,8 @@ class ClickHouseCluster: self.with_zookeeper = True self.base_cmd.extend(["--file", zookeeper_docker_compose_path]) self.base_zookeeper_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1101,7 +1103,8 @@ class ClickHouseCluster: self.with_zookeeper = True self.base_cmd.extend(["--file", keeper_docker_compose_path]) self.base_zookeeper_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1120,7 +1123,8 @@ class ClickHouseCluster: ] ) self.base_mysql_client_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1144,7 +1148,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_mysql.yml")] ) self.base_mysql57_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1168,7 +1173,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_mysql_8_0.yml")] ) self.base_mysql8_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1194,7 +1200,8 @@ class ClickHouseCluster: ] ) self.base_mysql_cluster_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1215,7 +1222,8 @@ class ClickHouseCluster: self.with_postgres = True self.base_postgres_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1241,7 +1249,8 @@ class ClickHouseCluster: ] ) self.base_postgres_cluster_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1263,7 +1272,8 @@ class ClickHouseCluster: ] ) self.base_postgresql_java_client_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1283,7 +1293,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_hdfs.yml")] ) self.base_hdfs_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1311,7 +1322,8 @@ class ClickHouseCluster: ] ) self.base_kerberized_hdfs_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1334,7 +1346,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_kafka.yml")] ) self.base_kafka_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1360,7 +1373,8 @@ class ClickHouseCluster: ] ) self.base_kerberized_kafka_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1381,7 +1395,8 @@ class ClickHouseCluster: ] ) self.base_kerberos_kdc_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1401,7 +1416,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_redis.yml")] ) self.base_redis_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1425,7 +1441,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_rabbitmq.yml")] ) self.base_rabbitmq_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1446,7 +1463,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_nats.yml")] ) self.base_nats_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1472,7 +1490,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_mongo.yml")] ) self.base_mongo_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1490,7 +1509,8 @@ class ClickHouseCluster: ) self.base_coredns_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1513,7 +1533,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_minio.yml")] ) self.base_minio_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1539,7 +1560,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_azurite.yml")] ) self.base_azurite_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1556,7 +1578,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_cassandra.yml")] ) self.base_cassandra_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1573,7 +1596,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_ldap.yml")] ) self.base_ldap_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1591,7 +1615,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_jdbc_bridge.yml")] ) self.base_jdbc_bridge_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1608,7 +1633,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_nginx.yml")] ) self.base_nginx_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1624,7 +1650,8 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_hive.yml")] ) self.base_hive_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -1660,7 +1687,8 @@ class ClickHouseCluster: ] ) self.base_prometheus_cmd = [ - "docker-compose", + "docker", + "compose", "--env-file", instance.env_file, "--project-name", @@ -2069,7 +2097,7 @@ class ClickHouseCluster: def get_instance_docker_id(self, instance_name): # According to how docker-compose names containers. - return self.project_name + "_" + instance_name + "_1" + return self.project_name + "-" + instance_name + "-1" def _replace(self, path, what, to): with open(path, "r") as p: @@ -3250,7 +3278,7 @@ class ClickHouseCluster: ) else: logging.warning( - "docker-compose up was not called. Trying to export docker.log for running containers" + "docker compose up was not called. Trying to export docker.log for running containers" ) self.cleanup() diff --git a/tests/integration/runner b/tests/integration/runner index 83c28cff7c7..9406e9a0286 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -485,7 +485,7 @@ if __name__ == "__main__": cmd = cmd_base + " " + args.command cmd_pre_pull = ( f"{cmd_base} find /ClickHouse/tests/integration/compose -name docker_compose_*.yml " - r"-exec docker-compose -f '{}' pull \;" + r"-exec docker compose -f '{}' pull \;" ) containers = subprocess.check_output( diff --git a/tests/integration/test_dotnet_client/test.py b/tests/integration/test_dotnet_client/test.py index 29ce9bab07c..86461eb6442 100644 --- a/tests/integration/test_dotnet_client/test.py +++ b/tests/integration/test_dotnet_client/test.py @@ -39,7 +39,8 @@ def dotnet_container(): ) run_and_check( [ - "docker-compose", + "docker", + "compose", "-p", cluster.project_name, "-f", diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index e84396ad300..29af28cbd25 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -52,7 +52,8 @@ def golang_container(): ) run_and_check( [ - "docker-compose", + "docker", + "compose", "-p", cluster.project_name, "-f", @@ -77,7 +78,8 @@ def php_container(): ) run_and_check( [ - "docker-compose", + "docker", + "compose", "--env-file", cluster.instances["node"].env_file, "-p", @@ -104,7 +106,8 @@ def nodejs_container(): ) run_and_check( [ - "docker-compose", + "docker", + "compose", "--env-file", cluster.instances["node"].env_file, "-p", @@ -131,7 +134,8 @@ def java_container(): ) run_and_check( [ - "docker-compose", + "docker", + "compose", "--env-file", cluster.instances["node"].env_file, "-p", From 9a2c8916c9730a08ade801e71041e6db1d4bcb38 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Oct 2024 08:55:29 +0200 Subject: [PATCH 37/65] Add return type to run_and_check --- tests/integration/helpers/cluster.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 87571b7115d..5ae39cb6ea3 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -18,6 +18,7 @@ import traceback import urllib.parse from functools import cache from pathlib import Path +from typing import List, Tuple import requests import urllib3 @@ -101,7 +102,7 @@ def run_and_check( timeout=300, nothrow=False, detach=False, -): +) -> str: if detach: subprocess.Popen( args, @@ -110,11 +111,17 @@ def run_and_check( env=env, shell=shell, ) - return + return "" logging.debug(f"Command:{args}") res = subprocess.run( - args, stdout=stdout, stderr=stderr, env=env, shell=shell, timeout=timeout + args, + stdout=stdout, + stderr=stderr, + env=env, + shell=shell, + timeout=timeout, + check=False, ) out = res.stdout.decode("utf-8", "ignore") err = res.stderr.decode("utf-8", "ignore") From 1fcbdb8b627ce9c6d4067ee736079254eda47508 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Oct 2024 08:57:31 +0200 Subject: [PATCH 38/65] Add docker_exec and ClickHouseCluster.compose_cmd to reduce boilerplate --- tests/integration/helpers/cluster.py | 273 +++++++-------------------- 1 file changed, 72 insertions(+), 201 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 5ae39cb6ea3..409b466ad1d 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -194,6 +194,11 @@ class PortPoolManager: self.used_ports.clear() +def docker_exec(*args: str) -> Tuple[str, ...]: + "Function to ease the `docker exec -i...`" + return ("docker", "exec", "-i", *args) + + def retry_exception(num, delay, func, exception=Exception, *args, **kwargs): """ Retry if `func()` throws, `num` times. @@ -251,10 +256,7 @@ def get_docker_compose_path(): def check_kafka_is_available(kafka_id, kafka_port): p = subprocess.Popen( - ( - "docker", - "exec", - "-i", + docker_exec( kafka_id, "/usr/bin/kafka-broker-api-versions", "--bootstrap-server", @@ -269,14 +271,7 @@ def check_kafka_is_available(kafka_id, kafka_port): def check_kerberos_kdc_is_available(kerberos_kdc_id): p = subprocess.Popen( - ( - "docker", - "exec", - "-i", - kerberos_kdc_id, - "/etc/rc.d/init.d/krb5kdc", - "status", - ), + docker_exec(kerberos_kdc_id, "/etc/rc.d/init.d/krb5kdc", "status"), stdout=subprocess.PIPE, stderr=subprocess.PIPE, ) @@ -286,7 +281,7 @@ def check_kerberos_kdc_is_available(kerberos_kdc_id): def check_postgresql_java_client_is_available(postgresql_java_client_id): p = subprocess.Popen( - ("docker", "exec", "-i", postgresql_java_client_id, "java", "-version"), + docker_exec(postgresql_java_client_id, "java", "-version"), stdout=subprocess.PIPE, ) p.communicate() @@ -295,12 +290,9 @@ def check_postgresql_java_client_is_available(postgresql_java_client_id): def check_rabbitmq_is_available(rabbitmq_id, cookie): p = subprocess.Popen( - ( - "docker", - "exec", + docker_exec( "-e", f"RABBITMQ_ERLANG_COOKIE={cookie}", - "-i", rabbitmq_id, "rabbitmqctl", "await_startup", @@ -313,12 +305,9 @@ def check_rabbitmq_is_available(rabbitmq_id, cookie): def rabbitmq_debuginfo(rabbitmq_id, cookie): p = subprocess.Popen( - ( - "docker", - "exec", + docker_exec( "-e", f"RABBITMQ_ERLANG_COOKIE={cookie}", - "-i", rabbitmq_id, "rabbitmq-diagnostics", "status", @@ -328,12 +317,9 @@ def rabbitmq_debuginfo(rabbitmq_id, cookie): p.communicate() p = subprocess.Popen( - ( - "docker", - "exec", + docker_exec( "-e", f"RABBITMQ_ERLANG_COOKIE={cookie}", - "-i", rabbitmq_id, "rabbitmq-diagnostics", "listeners", @@ -343,12 +329,9 @@ def rabbitmq_debuginfo(rabbitmq_id, cookie): p.communicate() p = subprocess.Popen( - ( - "docker", - "exec", + docker_exec( "-e", f"RABBITMQ_ERLANG_COOKIE={cookie}", - "-i", rabbitmq_id, "rabbitmq-diagnostics", "environment", @@ -383,12 +366,9 @@ async def nats_connect_ssl(nats_port, user, password, ssl_ctx=None): def enable_consistent_hash_plugin(rabbitmq_id, cookie): p = subprocess.Popen( - ( - "docker", - "exec", + docker_exec( "-e", f"RABBITMQ_ERLANG_COOKIE={cookie}", - "-i", rabbitmq_id, "rabbitmq-plugins", "enable", @@ -797,6 +777,9 @@ class ClickHouseCluster: self.port_pool = PortPoolManager() + def compose_cmd(self, *args: str) -> List[str]: + return ["docker", "compose", "--project-name", self.project_name, *args] + @property def kafka_port(self): if self._kafka_port: @@ -1027,16 +1010,12 @@ class ClickHouseCluster: self.with_zookeeper_secure = True self.base_cmd.extend(["--file", zookeeper_docker_compose_path]) - self.base_zookeeper_cmd = [ - "docker", - "compose", + self.base_zookeeper_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", zookeeper_docker_compose_path, - ] + ) return self.base_zookeeper_cmd def setup_zookeeper_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1060,16 +1039,12 @@ class ClickHouseCluster: self.with_zookeeper = True self.base_cmd.extend(["--file", zookeeper_docker_compose_path]) - self.base_zookeeper_cmd = [ - "docker", - "compose", + self.base_zookeeper_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", zookeeper_docker_compose_path, - ] + ) return self.base_zookeeper_cmd def setup_keeper_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1109,16 +1084,12 @@ class ClickHouseCluster: self.with_zookeeper = True self.base_cmd.extend(["--file", keeper_docker_compose_path]) - self.base_zookeeper_cmd = [ - "docker", - "compose", + self.base_zookeeper_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", keeper_docker_compose_path, - ] + ) return self.base_zookeeper_cmd def setup_mysql_client_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1129,16 +1100,12 @@ class ClickHouseCluster: p.join(docker_compose_yml_dir, "docker_compose_mysql_client.yml"), ] ) - self.base_mysql_client_cmd = [ - "docker", - "compose", + self.base_mysql_client_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_mysql_client.yml"), - ] + ) return self.base_mysql_client_cmd @@ -1154,16 +1121,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_mysql.yml")] ) - self.base_mysql57_cmd = [ - "docker", - "compose", + self.base_mysql57_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_mysql.yml"), - ] + ) return self.base_mysql57_cmd @@ -1179,16 +1142,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_mysql_8_0.yml")] ) - self.base_mysql8_cmd = [ - "docker", - "compose", + self.base_mysql8_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_mysql_8_0.yml"), - ] + ) return self.base_mysql8_cmd @@ -1206,16 +1165,12 @@ class ClickHouseCluster: p.join(docker_compose_yml_dir, "docker_compose_mysql_cluster.yml"), ] ) - self.base_mysql_cluster_cmd = [ - "docker", - "compose", + self.base_mysql_cluster_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_mysql_cluster.yml"), - ] + ) return self.base_mysql_cluster_cmd @@ -1228,16 +1183,12 @@ class ClickHouseCluster: env_variables["POSTGRES_LOGS_FS"] = "bind" self.with_postgres = True - self.base_postgres_cmd = [ - "docker", - "compose", + self.base_postgres_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_postgres.yml"), - ] + ) return self.base_postgres_cmd def setup_postgres_cluster_cmd( @@ -1255,16 +1206,12 @@ class ClickHouseCluster: p.join(docker_compose_yml_dir, "docker_compose_postgres_cluster.yml"), ] ) - self.base_postgres_cluster_cmd = [ - "docker", - "compose", + self.base_postgres_cluster_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_postgres_cluster.yml"), - ] + ) def setup_postgresql_java_client_cmd( self, instance, env_variables, docker_compose_yml_dir @@ -1278,16 +1225,12 @@ class ClickHouseCluster: ), ] ) - self.base_postgresql_java_client_cmd = [ - "docker", - "compose", + self.base_postgresql_java_client_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_postgresql_java_client.yml"), - ] + ) def setup_hdfs_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_hdfs = True @@ -1299,16 +1242,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_hdfs.yml")] ) - self.base_hdfs_cmd = [ - "docker", - "compose", + self.base_hdfs_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_hdfs.yml"), - ] + ) logging.debug("HDFS BASE CMD:{self.base_hdfs_cmd)}") return self.base_hdfs_cmd @@ -1328,16 +1267,12 @@ class ClickHouseCluster: p.join(docker_compose_yml_dir, "docker_compose_kerberized_hdfs.yml"), ] ) - self.base_kerberized_hdfs_cmd = [ - "docker", - "compose", + self.base_kerberized_hdfs_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_kerberized_hdfs.yml"), - ] + ) return self.base_kerberized_hdfs_cmd def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1352,16 +1287,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_kafka.yml")] ) - self.base_kafka_cmd = [ - "docker", - "compose", + self.base_kafka_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_kafka.yml"), - ] + ) return self.base_kafka_cmd def setup_kerberized_kafka_cmd( @@ -1379,16 +1310,12 @@ class ClickHouseCluster: p.join(docker_compose_yml_dir, "docker_compose_kerberized_kafka.yml"), ] ) - self.base_kerberized_kafka_cmd = [ - "docker", - "compose", + self.base_kerberized_kafka_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_kerberized_kafka.yml"), - ] + ) return self.base_kerberized_kafka_cmd def setup_kerberos_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1401,16 +1328,12 @@ class ClickHouseCluster: p.join(docker_compose_yml_dir, "docker_compose_kerberos_kdc.yml"), ] ) - self.base_kerberos_kdc_cmd = [ - "docker", - "compose", + self.base_kerberos_kdc_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_kerberos_kdc.yml"), - ] + ) return self.base_kerberos_kdc_cmd def setup_redis_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1422,16 +1345,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_redis.yml")] ) - self.base_redis_cmd = [ - "docker", - "compose", + self.base_redis_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_redis.yml"), - ] + ) return self.base_redis_cmd def setup_rabbitmq_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1447,16 +1366,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_rabbitmq.yml")] ) - self.base_rabbitmq_cmd = [ - "docker", - "compose", + self.base_rabbitmq_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_rabbitmq.yml"), - ] + ) return self.base_rabbitmq_cmd def setup_nats_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1469,16 +1384,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_nats.yml")] ) - self.base_nats_cmd = [ - "docker", - "compose", + self.base_nats_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_nats.yml"), - ] + ) return self.base_nats_cmd def setup_mongo_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1496,16 +1407,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_mongo.yml")] ) - self.base_mongo_cmd = [ - "docker", - "compose", + self.base_mongo_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_mongo.yml"), - ] + ) return self.base_mongo_cmd def setup_coredns_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1515,16 +1422,12 @@ class ClickHouseCluster: ["--file", p.join(docker_compose_yml_dir, "docker_compose_coredns.yml")] ) - self.base_coredns_cmd = [ - "docker", - "compose", + self.base_coredns_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_coredns.yml"), - ] + ) return self.base_coredns_cmd @@ -1539,16 +1442,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_minio.yml")] ) - self.base_minio_cmd = [ - "docker", - "compose", + self.base_minio_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_minio.yml"), - ] + ) return self.base_minio_cmd def setup_azurite_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1566,16 +1465,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_azurite.yml")] ) - self.base_azurite_cmd = [ - "docker", - "compose", + self.base_azurite_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_azurite.yml"), - ] + ) return self.base_azurite_cmd def setup_cassandra_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1584,16 +1479,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_cassandra.yml")] ) - self.base_cassandra_cmd = [ - "docker", - "compose", + self.base_cassandra_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_cassandra.yml"), - ] + ) return self.base_cassandra_cmd def setup_ldap_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1602,16 +1493,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_ldap.yml")] ) - self.base_ldap_cmd = [ - "docker", - "compose", + self.base_ldap_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_ldap.yml"), - ] + ) return self.base_ldap_cmd def setup_jdbc_bridge_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1621,16 +1508,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_jdbc_bridge.yml")] ) - self.base_jdbc_bridge_cmd = [ - "docker", - "compose", + self.base_jdbc_bridge_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_jdbc_bridge.yml"), - ] + ) return self.base_jdbc_bridge_cmd def setup_nginx_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1639,16 +1522,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_nginx.yml")] ) - self.base_nginx_cmd = [ - "docker", - "compose", + self.base_nginx_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_nginx.yml"), - ] + ) return self.base_nginx_cmd def setup_hive(self, instance, env_variables, docker_compose_yml_dir): @@ -1656,16 +1535,12 @@ class ClickHouseCluster: self.base_cmd.extend( ["--file", p.join(docker_compose_yml_dir, "docker_compose_hive.yml")] ) - self.base_hive_cmd = [ - "docker", - "compose", + self.base_hive_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_hive.yml"), - ] + ) return self.base_hive_cmd def setup_prometheus_cmd(self, instance, env_variables, docker_compose_yml_dir): @@ -1693,16 +1568,12 @@ class ClickHouseCluster: p.join(docker_compose_yml_dir, "docker_compose_prometheus.yml"), ] ) - self.base_prometheus_cmd = [ - "docker", - "compose", + self.base_prometheus_cmd = self.compose_cmd( "--env-file", instance.env_file, - "--project-name", - self.project_name, "--file", p.join(docker_compose_yml_dir, "docker_compose_prometheus.yml"), - ] + ) return self.base_prometheus_cmd def add_instance( From 93519c5a29d0c5dc8fa340c855fdcb91bec29abd Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Oct 2024 11:47:17 +0200 Subject: [PATCH 39/65] Fix compose v1 naming scheme --- tests/integration/test_dotnet_client/test.py | 10 ++--- tests/integration/test_mysql_protocol/test.py | 38 ++++++------------- 2 files changed, 15 insertions(+), 33 deletions(-) diff --git a/tests/integration/test_dotnet_client/test.py b/tests/integration/test_dotnet_client/test.py index 86461eb6442..c74f55c7b1a 100644 --- a/tests/integration/test_dotnet_client/test.py +++ b/tests/integration/test_dotnet_client/test.py @@ -38,20 +38,16 @@ def dotnet_container(): DOCKER_COMPOSE_PATH, "docker_compose_dotnet_client.yml" ) run_and_check( - [ - "docker", - "compose", - "-p", - cluster.project_name, + cluster.compose_cmd( "-f", docker_compose, "up", "--force-recreate", "-d", "--no-build", - ] + ) ) - yield docker.from_env().containers.get(cluster.project_name + "_dotnet1_1") + yield docker.from_env().containers.get(cluster.get_instance_docker_id("dotnet1")) def test_dotnet_client(started_cluster, dotnet_container): diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 29af28cbd25..cfadc27a2b5 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -51,9 +51,7 @@ def golang_container(): DOCKER_COMPOSE_PATH, "docker_compose_mysql_golang_client.yml" ) run_and_check( - [ - "docker", - "compose", + cluster.compose_cmd( "-p", cluster.project_name, "-f", @@ -62,13 +60,13 @@ def golang_container(): "--force-recreate", "-d", "--no-build", - ] + ) ) yield docker.DockerClient( base_url="unix:///var/run/docker.sock", version=cluster.docker_api_version, timeout=600, - ).containers.get(cluster.project_name + "_golang1_1") + ).containers.get(cluster.get_instance_docker_id("golang1")) @pytest.fixture(scope="module") @@ -77,26 +75,22 @@ def php_container(): DOCKER_COMPOSE_PATH, "docker_compose_mysql_php_client.yml" ) run_and_check( - [ - "docker", - "compose", + cluster.compose_cmd( "--env-file", cluster.instances["node"].env_file, - "-p", - cluster.project_name, "-f", docker_compose, "up", "--force-recreate", "-d", "--no-build", - ] + ) ) yield docker.DockerClient( base_url="unix:///var/run/docker.sock", version=cluster.docker_api_version, timeout=600, - ).containers.get(cluster.project_name + "_php1_1") + ).containers.get(cluster.get_instance_docker_id("php1")) @pytest.fixture(scope="module") @@ -105,26 +99,22 @@ def nodejs_container(): DOCKER_COMPOSE_PATH, "docker_compose_mysql_js_client.yml" ) run_and_check( - [ - "docker", - "compose", + cluster.compose_cmd( "--env-file", cluster.instances["node"].env_file, - "-p", - cluster.project_name, "-f", docker_compose, "up", "--force-recreate", "-d", "--no-build", - ] + ) ) yield docker.DockerClient( base_url="unix:///var/run/docker.sock", version=cluster.docker_api_version, timeout=600, - ).containers.get(cluster.project_name + "_mysqljs1_1") + ).containers.get(cluster.get_instance_docker_id("mysqljs1")) @pytest.fixture(scope="module") @@ -133,26 +123,22 @@ def java_container(): DOCKER_COMPOSE_PATH, "docker_compose_mysql_java_client.yml" ) run_and_check( - [ - "docker", - "compose", + cluster.compose_cmd( "--env-file", cluster.instances["node"].env_file, - "-p", - cluster.project_name, "-f", docker_compose, "up", "--force-recreate", "-d", "--no-build", - ] + ) ) yield docker.DockerClient( base_url="unix:///var/run/docker.sock", version=cluster.docker_api_version, timeout=600, - ).containers.get(cluster.project_name + "_java1_1") + ).containers.get(cluster.get_instance_docker_id("java1")) def test_mysql_client(started_cluster): From 4caa22e8e5db8df47b1062902652ce7d8372cb51 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 1 Oct 2024 11:55:19 +0200 Subject: [PATCH 40/65] Update 02477_age_date32.sql --- tests/queries/0_stateless/02477_age_date32.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02477_age_date32.sql b/tests/queries/0_stateless/02477_age_date32.sql index 302c8870820..aa913a8d139 100644 --- a/tests/queries/0_stateless/02477_age_date32.sql +++ b/tests/queries/0_stateless/02477_age_date32.sql @@ -100,4 +100,5 @@ SELECT age('day', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), materia SELECT age('day', materialize(toDate32('2015-08-18', 'UTC')), materialize(toDate('2015-08-19', 'UTC')), 'UTC'); SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDate32('2015-08-19', 'UTC')), 'UTC'); +-- UBsan issue detected by fuzzer SELECT age('minute', toDate32(1234567890123456, 'UTC'), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'), 'UTC') From 40ebc5cd793a5fc990abeb9dbe9e188e1c703e11 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 1 Oct 2024 11:55:53 +0200 Subject: [PATCH 41/65] Update 02477_age_date32.reference --- tests/queries/0_stateless/02477_age_date32.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02477_age_date32.reference b/tests/queries/0_stateless/02477_age_date32.reference index 02505439b12..901569dcc5a 100644 --- a/tests/queries/0_stateless/02477_age_date32.reference +++ b/tests/queries/0_stateless/02477_age_date32.reference @@ -167,6 +167,7 @@ SELECT age('day', materialize(toDate32('2015-08-18', 'UTC')), materialize(toDate 1 SELECT age('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDate32('2015-08-19', 'UTC')), 'UTC'); 1 +-- UBsan issue detected by fuzzer SELECT age('minute', toDate32(1234567890123456, 'UTC'), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'), 'UTC') -196176960 From 719258a612162c7707b23b9639d2dc4234736457 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 1 Oct 2024 12:48:04 +0200 Subject: [PATCH 42/65] Update test.py --- tests/integration/test_storage_s3_queue/test.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 5e93c991aed..91dd698ff49 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -4,7 +4,6 @@ import logging import random import string import time -from uuid import uuid4 import pytest @@ -981,7 +980,7 @@ def test_max_set_age(started_cluster): ).encode() # use a different filename for each test to allow running a bunch of them sequentially with --count - file_with_error = f"max_set_age_fail_{uuid4().hex[:8]}.csv" + file_with_error = f"max_set_age_fail_{uuid.uuid4().hex[:8]}.csv" put_s3_file_content(started_cluster, f"{files_path}/{file_with_error}", values_csv) wait_for_condition(lambda: failed_count + 1 == get_object_storage_failures()) @@ -1913,7 +1912,7 @@ def test_commit_on_limit(started_cluster): def test_upgrade_2(started_cluster): node = started_cluster.instances["instance_24.5"] - table_name = f"test_upgrade_2_{uuid4().hex[:8]}" + table_name = f"test_upgrade_2_{uuid.uuid4().hex[:8]}" dst_table_name = f"{table_name}_dst" # A unique path is necessary for repeatable tests keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" From f62c2ea4d8294310ac225107ddc2d73ed850432f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 1 Oct 2024 12:50:20 +0200 Subject: [PATCH 43/65] enhance SettingsChangesHistory --- src/Core/SettingsChangesHistory.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 6757dd84b2e..43f8d37cee7 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -72,6 +72,8 @@ static std::initializer_list Date: Tue, 1 Oct 2024 12:55:03 +0200 Subject: [PATCH 44/65] Update test.py --- tests/integration/test_storage_s3_queue/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 91dd698ff49..14d4fb3cd31 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -1915,7 +1915,7 @@ def test_upgrade_2(started_cluster): table_name = f"test_upgrade_2_{uuid.uuid4().hex[:8]}" dst_table_name = f"{table_name}_dst" # A unique path is necessary for repeatable tests - keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" + keeper_path = f"/clickhouse/test_{table_name}}" files_path = f"{table_name}_data" files_to_generate = 10 From 94c434086d2cb9c2c309ebac79ba2ac81e78cc85 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 1 Oct 2024 13:02:31 +0200 Subject: [PATCH 45/65] Fix --- tests/integration/test_storage_s3_queue/test.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 14d4fb3cd31..8a7ff608bc8 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -6,6 +6,7 @@ import string import time import pytest +import uuid from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster, ClickHouseInstance @@ -1915,7 +1916,7 @@ def test_upgrade_2(started_cluster): table_name = f"test_upgrade_2_{uuid.uuid4().hex[:8]}" dst_table_name = f"{table_name}_dst" # A unique path is necessary for repeatable tests - keeper_path = f"/clickhouse/test_{table_name}}" + keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" files_to_generate = 10 @@ -1956,9 +1957,9 @@ def test_replicated(started_cluster): node1 = started_cluster.instances["node1"] node2 = started_cluster.instances["node2"] - table_name = f"test_replicated" + table_name = f"test_replicated_{uuid.uuid4().hex[:8]}" dst_table_name = f"{table_name}_dst" - keeper_path = f"/clickhouse/test_{table_name}_{generate_random_string()}" + keeper_path = f"/clickhouse/test_{table_name}" files_path = f"{table_name}_data" files_to_generate = 1000 From 72d1c8e36df6cdf86d29ecaf7c4c544ebb633825 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 1 Oct 2024 11:08:31 +0000 Subject: [PATCH 46/65] Automatic style fix --- tests/integration/test_storage_s3_queue/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3_queue/test.py b/tests/integration/test_storage_s3_queue/test.py index 8a7ff608bc8..c235e5dad89 100644 --- a/tests/integration/test_storage_s3_queue/test.py +++ b/tests/integration/test_storage_s3_queue/test.py @@ -4,9 +4,9 @@ import logging import random import string import time +import uuid import pytest -import uuid from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster, ClickHouseInstance From a09d21736691737093208af1ddd6c3de7acc5074 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Tue, 1 Oct 2024 13:43:59 +0200 Subject: [PATCH 47/65] Update 03246_json_subcolumn_correct_type.sql --- tests/queries/0_stateless/03246_json_subcolumn_correct_type.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03246_json_subcolumn_correct_type.sql b/tests/queries/0_stateless/03246_json_subcolumn_correct_type.sql index c21228ec33f..ac84bb20dda 100644 --- a/tests/queries/0_stateless/03246_json_subcolumn_correct_type.sql +++ b/tests/queries/0_stateless/03246_json_subcolumn_correct_type.sql @@ -1,4 +1,5 @@ set allow_experimental_json_type=1; +set enable_analyzer=1; drop table if exists test; create table test (json JSON(max_dynamic_types=1)) engine=Memory; insert into test values ('{"c0" : 1}'), ('{"c0" : 2}'); From f6ca5cee9d5a3710b74b0bf5e0af3d652f18ad81 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 1 Oct 2024 17:27:58 +0200 Subject: [PATCH 48/65] Update 01079_parallel_alter_add_drop_column_zookeeper.sh --- .../01079_parallel_alter_add_drop_column_zookeeper.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh index f05a0fed965..eee298eb07d 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh @@ -107,10 +107,10 @@ check_replication_consistency "concurrent_alter_add_drop_" "count(), sum(key), s for i in $(seq $REPLICAS); do $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_add_drop_$i" - $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.mutations WHERE is_done = 0 and table = 'concurrent_alter_add_drop_$i'" - $CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE is_done = 0 and table = 'concurrent_alter_add_drop_$i'" - $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.replication_queue WHERE table = 'concurrent_alter_add_drop_$i'" - $CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue WHERE table = 'concurrent_alter_add_drop_$i' and (type = 'ALTER_METADATA' or type = 'MUTATE_PART')" + $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.mutations WHERE is_done = 0 and table = 'concurrent_alter_add_drop_$i' and database='$CLICKHOUSE_DATABASE'" + $CLICKHOUSE_CLIENT --query "SELECT * FROM system.mutations WHERE is_done = 0 and table = 'concurrent_alter_add_drop_$i' and database='$CLICKHOUSE_DATABASE'" + $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM system.replication_queue WHERE table = 'concurrent_alter_add_drop_$i' and database='$CLICKHOUSE_DATABASE'" + $CLICKHOUSE_CLIENT --query "SELECT * FROM system.replication_queue WHERE table = 'concurrent_alter_add_drop_$i' and (type = 'ALTER_METADATA' or type = 'MUTATE_PART') and database='$CLICKHOUSE_DATABASE'" $CLICKHOUSE_CLIENT --query "DETACH TABLE concurrent_alter_add_drop_$i" $CLICKHOUSE_CLIENT --query "ATTACH TABLE concurrent_alter_add_drop_$i" From 998fb7e341a0ab14583af8c3d6ca803c52b219d8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Oct 2024 17:36:39 +0200 Subject: [PATCH 49/65] Make JobConfig.run_by_label a set run_by_labels --- tests/ci/ci.py | 2 +- tests/ci/ci_config.py | 12 ++++++------ tests/ci/ci_definitions.py | 3 ++- tests/ci/ci_settings.py | 6 +++--- tests/ci/test_ci_config.py | 6 +++--- tests/ci/test_ci_options.py | 12 +++++------- 6 files changed, 20 insertions(+), 21 deletions(-) diff --git a/tests/ci/ci.py b/tests/ci/ci.py index 7cc32fa19c5..04f5a1625d1 100644 --- a/tests/ci/ci.py +++ b/tests/ci/ci.py @@ -427,7 +427,7 @@ def _mark_success_action( # do nothing, exit without failure print(f"ERROR: no status file for job [{job}]") - if job_config.run_by_label or not job_config.has_digest(): + if job_config.run_by_labels or not job_config.has_digest(): print(f"Job [{job}] has no digest or run by label in CI - do not cache") else: if pr_info.is_master: diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 411cd87c471..ea6289bd9d5 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -261,7 +261,7 @@ class CI: compiler="clang-18", package_type="fuzzers", ), - run_by_label=Tags.libFuzzer, + run_by_labels=[Tags.libFuzzer], ), JobNames.BUILD_CHECK: CommonJobConfigs.BUILD_REPORT.with_properties(), JobNames.INSTALL_TEST_AMD: CommonJobConfigs.INSTALL_TEST.with_properties( @@ -479,13 +479,13 @@ class CI: ), JobNames.JEPSEN_KEEPER: JobConfig( required_builds=[BuildNames.BINARY_RELEASE], - run_by_label="jepsen-test", + run_by_labels=[Labels.JEPSEN_TEST], run_command="jepsen_check.py keeper", runner_type=Runners.STYLE_CHECKER_ARM, ), JobNames.JEPSEN_SERVER: JobConfig( required_builds=[BuildNames.BINARY_RELEASE], - run_by_label="jepsen-test", + run_by_labels=[Labels.JEPSEN_TEST], run_command="jepsen_check.py server", runner_type=Runners.STYLE_CHECKER_ARM, ), @@ -495,7 +495,7 @@ class CI: JobNames.PERFORMANCE_TEST_ARM64: CommonJobConfigs.PERF_TESTS.with_properties( required_builds=[BuildNames.PACKAGE_AARCH64], num_batches=4, - run_by_label="pr-performance", + run_by_labels=[Labels.PR_PERFORMANCE], runner_type=Runners.FUNC_TESTER_ARM, ), JobNames.SQLANCER: CommonJobConfigs.SQLLANCER_TEST.with_properties( @@ -520,7 +520,7 @@ class CI: ), JobNames.LIBFUZZER_TEST: JobConfig( required_builds=[BuildNames.FUZZERS], - run_by_label=Tags.libFuzzer, + run_by_labels=[Tags.libFuzzer], timeout=10800, run_command='libfuzzer_test_check.py "$CHECK_NAME"', runner_type=Runners.STYLE_CHECKER, @@ -557,7 +557,7 @@ class CI: runner_type=Runners.STYLE_CHECKER_ARM, ), JobNames.BUGFIX_VALIDATE: JobConfig( - run_by_label="pr-bugfix", + run_by_labels=[Labels.PR_BUGFIX], run_command="bugfix_validate_check.py", timeout=2400, runner_type=Runners.STYLE_CHECKER, diff --git a/tests/ci/ci_definitions.py b/tests/ci/ci_definitions.py index c62727e4e2b..26f17ed56c1 100644 --- a/tests/ci/ci_definitions.py +++ b/tests/ci/ci_definitions.py @@ -22,6 +22,7 @@ class Labels: PR_CHERRYPICK = "pr-cherrypick" PR_CI = "pr-ci" PR_FEATURE = "pr-feature" + PR_PERFORMANCE = "pr-performance" PR_SYNCED_TO_CLOUD = "pr-synced-to-cloud" PR_SYNC_UPSTREAM = "pr-sync-upstream" RELEASE = "release" @@ -335,7 +336,7 @@ class JobConfig: # sets number of batches for a multi-batch job num_batches: int = 1 # label that enables job in CI, if set digest isn't used - run_by_label: str = "" + run_by_labels: List[str] = field(default_factory=list) # to run always regardless of the job digest or/and label run_always: bool = False # disables CI await for a given job diff --git a/tests/ci/ci_settings.py b/tests/ci/ci_settings.py index ba406834568..bfff9abceb6 100644 --- a/tests/ci/ci_settings.py +++ b/tests/ci/ci_settings.py @@ -151,10 +151,10 @@ class CiSettings: return True return False - if job_config.run_by_label: - if job_config.run_by_label in labels and is_pr: + if job_config.run_by_labels: + if set(job_config.run_by_labels).intersection(labels) and is_pr: print( - f"Job [{job}] selected by GH label [{job_config.run_by_label}] - pass" + f"Job [{job}] selected by GH label [{job_config.run_by_labels}] - pass" ) return True return False diff --git a/tests/ci/test_ci_config.py b/tests/ci/test_ci_config.py index c6689f102ea..fc280a426e5 100644 --- a/tests/ci/test_ci_config.py +++ b/tests/ci/test_ci_config.py @@ -304,7 +304,7 @@ class TestCIConfig(unittest.TestCase): for job, config in CI.JOB_CONFIGS.items(): if ( CI.is_build_job(job) - and not config.run_by_label + and not config.run_by_labels and job not in expected_jobs_to_do ): # expected to run all builds jobs @@ -358,7 +358,7 @@ class TestCIConfig(unittest.TestCase): continue if config.release_only: continue - if config.run_by_label: + if config.run_by_labels: continue expected_jobs_to_do.append(job) @@ -391,7 +391,7 @@ class TestCIConfig(unittest.TestCase): for job, config in CI.JOB_CONFIGS.items(): if config.pr_only: continue - if config.run_by_label: + if config.run_by_labels: continue if job in CI.MQ_JOBS: continue diff --git a/tests/ci/test_ci_options.py b/tests/ci/test_ci_options.py index 7889072df35..e2dc71de469 100644 --- a/tests/ci/test_ci_options.py +++ b/tests/ci/test_ci_options.py @@ -173,9 +173,8 @@ class TestCIOptions(unittest.TestCase): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs["fuzzers"].run_by_label = ( - "TEST_LABEL" # check "fuzzers" appears in the result due to the label - ) + # check "fuzzers" appears in the result due to the label + jobs_configs["fuzzers"].run_by_labels = ["TEST_LABEL"] jobs_configs["Integration tests (asan)"].release_only = ( True # still must be included as it's set with include keywords ) @@ -222,7 +221,7 @@ class TestCIOptions(unittest.TestCase): } jobs_configs["Style check"].release_only = True jobs_configs["Fast test"].pr_only = True - jobs_configs["fuzzers"].run_by_label = "TEST_LABEL" + jobs_configs["fuzzers"].run_by_labels = ["TEST_LABEL"] # no settings are set filtered_jobs = list( CiSettings().apply( @@ -311,9 +310,8 @@ class TestCIOptions(unittest.TestCase): job: CI.JobConfig(runner_type=CI.Runners.STYLE_CHECKER) for job in _TEST_JOB_LIST } - jobs_configs["fuzzers"].run_by_label = ( - "TEST_LABEL" # check "fuzzers" does not appears in the result - ) + # check "fuzzers" does not appears in the result + jobs_configs["fuzzers"].run_by_labels = ["TEST_LABEL"] jobs_configs["Integration tests (asan)"].release_only = True filtered_jobs = list( ci_options.apply( From 004f159548741b51984b6bc9451e6700733ae3a0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Oct 2024 17:37:09 +0200 Subject: [PATCH 50/65] Launch BUGFIX_VALIDATE job for a set of labels --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index ea6289bd9d5..a34ef624ce3 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -557,7 +557,7 @@ class CI: runner_type=Runners.STYLE_CHECKER_ARM, ), JobNames.BUGFIX_VALIDATE: JobConfig( - run_by_labels=[Labels.PR_BUGFIX], + run_by_labels=[Labels.PR_BUGFIX, Labels.PR_CRITICAL_BUGFIX], run_command="bugfix_validate_check.py", timeout=2400, runner_type=Runners.STYLE_CHECKER, From cedc1d6ed962581890f5270ec942952d3ecf70c8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 1 Oct 2024 18:35:18 +0200 Subject: [PATCH 51/65] Remove references of avgWeighted(Decimal) from the docs --- .../aggregate-functions/reference/avgweighted.md | 4 +--- .../aggregate-functions/reference/avgweighted.md | 3 +-- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md index 304d0407d98..773b42aebaa 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md @@ -19,9 +19,7 @@ avgWeighted(x, weight) - `weight` — Weights of the values. `x` and `weight` must both be -[Integer](../../../sql-reference/data-types/int-uint.md), -[floating-point](../../../sql-reference/data-types/float.md), or -[Decimal](../../../sql-reference/data-types/decimal.md), +[Integer](../../../sql-reference/data-types/int-uint.md) or [floating-point](../../../sql-reference/data-types/float.md), but may have different types. **Returned value** diff --git a/docs/zh/sql-reference/aggregate-functions/reference/avgweighted.md b/docs/zh/sql-reference/aggregate-functions/reference/avgweighted.md index 5dc8d06df4a..4a1a54ff83b 100644 --- a/docs/zh/sql-reference/aggregate-functions/reference/avgweighted.md +++ b/docs/zh/sql-reference/aggregate-functions/reference/avgweighted.md @@ -21,8 +21,7 @@ avgWeighted(x, weight) `x` 和 `weight` 的类型必须是 [整数](../../../sql-reference/data-types/int-uint.md), 或 -[浮点数](../../../sql-reference/data-types/float.md), 或 -[定点数](../../../sql-reference/data-types/decimal.md), +[浮点数](../../../sql-reference/data-types/float.md), 但是可以不一样。 **返回值** From eac13401a0e16275606e5f4cea8d2a2933627120 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 1 Oct 2024 19:02:10 +0200 Subject: [PATCH 52/65] Update mergetree_mutations.lib --- tests/queries/0_stateless/mergetree_mutations.lib | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/mergetree_mutations.lib b/tests/queries/0_stateless/mergetree_mutations.lib index 9eeea87b52d..4b4f97efc51 100644 --- a/tests/queries/0_stateless/mergetree_mutations.lib +++ b/tests/queries/0_stateless/mergetree_mutations.lib @@ -35,6 +35,7 @@ function wait_for_all_mutations() if [[ $i -eq 200 ]]; then echo "Timed out while waiting for mutation to execute!" + ${CLICKHOUSE_CLIENT} -q "SELECT * FROM system.mutations WHERE database='$database' AND table like '$table' AND is_done=0" fi sleep 0.3 From b40af9c460c27008d348a96d52fe1f93503a919c Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 1 Oct 2024 17:13:18 +0000 Subject: [PATCH 53/65] Respect setting allow_simdjson in JSON type parser --- src/DataTypes/DataTypeObject.cpp | 22 +++++++++++++------ .../03246_json_simd_rapid_parsers.reference | 2 ++ .../03246_json_simd_rapid_parsers.sh | 10 +++++++++ 3 files changed, 27 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/03246_json_simd_rapid_parsers.reference create mode 100755 tests/queries/0_stateless/03246_json_simd_rapid_parsers.sh diff --git a/src/DataTypes/DataTypeObject.cpp b/src/DataTypes/DataTypeObject.cpp index 3a834c700df..4596a4c701f 100644 --- a/src/DataTypes/DataTypeObject.cpp +++ b/src/DataTypes/DataTypeObject.cpp @@ -24,7 +24,8 @@ #if USE_SIMDJSON # include -#elif USE_RAPIDJSON +#endif +#if USE_RAPIDJSON # include #else # include @@ -36,6 +37,7 @@ namespace Setting { extern const SettingsBool allow_experimental_object_type; extern const SettingsBool use_json_alias_for_old_object_type; + extern const SettingsBool allow_simdjson; } namespace ErrorCodes @@ -127,12 +129,18 @@ SerializationPtr DataTypeObject::doGetDefaultSerialization() const { case SchemaFormat::JSON: #if USE_SIMDJSON - return std::make_shared>( - std::move(typed_path_serializations), - paths_to_skip, - path_regexps_to_skip, - buildJSONExtractTree(getPtr(), "JSON serialization")); -#elif USE_RAPIDJSON + auto context = CurrentThread::getQueryContext(); + if (!context) + context = Context::getGlobalContextInstance(); + if (context->getSettingsRef()[Setting::allow_simdjson]) + return std::make_shared>( + std::move(typed_path_serializations), + paths_to_skip, + path_regexps_to_skip, + buildJSONExtractTree(getPtr(), "JSON serialization")); +#endif + +#if USE_RAPIDJSON return std::make_shared>( std::move(typed_path_serializations), paths_to_skip, diff --git a/tests/queries/0_stateless/03246_json_simd_rapid_parsers.reference b/tests/queries/0_stateless/03246_json_simd_rapid_parsers.reference new file mode 100644 index 00000000000..51993f072d5 --- /dev/null +++ b/tests/queries/0_stateless/03246_json_simd_rapid_parsers.reference @@ -0,0 +1,2 @@ +2 +2 diff --git a/tests/queries/0_stateless/03246_json_simd_rapid_parsers.sh b/tests/queries/0_stateless/03246_json_simd_rapid_parsers.sh new file mode 100755 index 00000000000..2f0a2ff44a9 --- /dev/null +++ b/tests/queries/0_stateless/03246_json_simd_rapid_parsers.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 --stacktrace -q "select '{\"a\" : 4ab2}'::JSON" 2>&1 | grep -c -F "SimdJSON" +$CLICKHOUSE_LOCAL --allow_experimental_json_type=1 --allow_simdjson=0 --stacktrace -q "select '{\"a\" : 4ab2}'::JSON" 2>&1 | grep -c -F "RapidJSON" + + From 6b86d119d8989567f0c65912b70944689a2f6363 Mon Sep 17 00:00:00 2001 From: Peter Date: Wed, 2 Oct 2024 01:16:54 +0800 Subject: [PATCH 54/65] Improve toTypeName function usage --- .../functions/other-functions.md | 26 ++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index d17d05165e8..a84d1fb5010 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -207,7 +207,31 @@ If `NULL` is passed, then the function returns type `Nullable(Nothing)`, which c **Syntax** ```sql -toTypeName(x) +toTypeName(value) +``` + +**Arguments** + +- `value` — The value with any arbitrary. + +**Returned value** + +- `value` converted to the current data type name. [String](../data-types/string.md). + +**Example** + +Query: + +```sql +SELECT toTypeName(123); +``` + +Result: + +```response +┌─toTypeName(123)─┐ +│ UInt8 │ +└─────────────────┘ ``` ## blockSize {#blockSize} From 3c387726a3e4c79c4ab70e673d7b6a4751a3d80b Mon Sep 17 00:00:00 2001 From: Peter Date: Wed, 2 Oct 2024 01:31:07 +0800 Subject: [PATCH 55/65] Improve toISOYear function usage --- .../functions/date-time-functions.md | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index b2e5fe8dbac..6db3b8a7b88 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1974,6 +1974,38 @@ Result: Converts a date, or date with time, to a UInt16 number containing the ISO Year number. +**Syntax** + +```sql +toISOYear(value) +``` + +**Arguments** + +- `value` — The value with date or date with time. + +**Returned value** + +- `value` converted to the current ISO year number. [UInt16](../data-types/int-uint.md). + +**Example** + +Query: + +```sql +SELECT + toISOYear(toDate('2024/10/02')) as year1, + toISOYear(toDateTime('2024/10/02 01:30:00')) as year2 +``` + +Result: + +```response +┌─year1─┬─year2─┐ +│ 2024 │ 2024 │ +└───────┴───────┘ +``` + ## toISOWeek Converts a date, or date with time, to a UInt8 number containing the ISO Week number. From b0e2b12103aabcc7bea9e081c8298763ad58942f Mon Sep 17 00:00:00 2001 From: Peter Date: Wed, 2 Oct 2024 01:49:18 +0800 Subject: [PATCH 56/65] Improve toIPv4OrDefault function usage --- .../functions/ip-address-functions.md | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/docs/en/sql-reference/functions/ip-address-functions.md b/docs/en/sql-reference/functions/ip-address-functions.md index 11a7749b33d..9416036aff1 100644 --- a/docs/en/sql-reference/functions/ip-address-functions.md +++ b/docs/en/sql-reference/functions/ip-address-functions.md @@ -280,6 +280,38 @@ SELECT Same as `toIPv4`, but if the IPv4 address has an invalid format, it returns `0.0.0.0` (0 IPv4). +**Syntax** + +```sql +toIPv4OrDefault(value) +``` + +**Arguments** + +- `value` — The value with IPv4 address. + +**Returned value** + +- `value` converted to the current IPv4 address. [String](../data-types/string.md). + +**Example** + +Query: + +```sql +SELECT + toIPv4OrDefault('192.168.0.1') AS s1, + toIPv4OrDefault('192.168.0') AS s2 +``` + +Result: + +```response +┌─s1──────────┬─s2──────┐ +│ 192.168.0.1 │ 0.0.0.0 │ +└─────────────┴─────────┘ +``` + ## toIPv4OrNull(string) Same as `toIPv4`, but if the IPv4 address has an invalid format, it returns null. From bd71f2b74c4da2a4a4956743795b18165d0da9a4 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Tue, 1 Oct 2024 19:57:48 +0200 Subject: [PATCH 57/65] Update 03246_json_simd_rapid_parsers.sh --- tests/queries/0_stateless/03246_json_simd_rapid_parsers.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03246_json_simd_rapid_parsers.sh b/tests/queries/0_stateless/03246_json_simd_rapid_parsers.sh index 2f0a2ff44a9..58299fa66bb 100755 --- a/tests/queries/0_stateless/03246_json_simd_rapid_parsers.sh +++ b/tests/queries/0_stateless/03246_json_simd_rapid_parsers.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 8b40f15c7b5710cf4612245198a6bcfdca44d57c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Oct 2024 15:28:28 +0200 Subject: [PATCH 58/65] Set docker_compose_net.yml only once, remove `version` from compose template --- tests/integration/helpers/cluster.py | 26 ++++++++++++++------------ 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 409b466ad1d..8b94aec1972 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1748,13 +1748,15 @@ class ClickHouseCluster: ) docker_compose_yml_dir = get_docker_compose_path() + docker_compose_net = p.join(docker_compose_yml_dir, "docker_compose_net.yml") self.instances[name] = instance - if ipv4_address is not None or ipv6_address is not None: + if not self.with_net_trics and ( + ipv4_address is not None or ipv6_address is not None + ): + # docker compose v2 does not accept more than one argument `-f net.yml` self.with_net_trics = True - self.base_cmd.extend( - ["--file", p.join(docker_compose_yml_dir, "docker_compose_net.yml")] - ) + self.base_cmd.extend(["--file", docker_compose_net]) self.base_cmd.extend(["--file", instance.docker_compose_path]) @@ -1893,12 +1895,6 @@ class ClickHouseCluster: self.setup_coredns_cmd(instance, env_variables, docker_compose_yml_dir) ) - if self.with_net_trics: - for cmd in cmds: - cmd.extend( - ["--file", p.join(docker_compose_yml_dir, "docker_compose_net.yml")] - ) - if with_redis and not self.with_redis: cmds.append( self.setup_redis_cmd(instance, env_variables, docker_compose_yml_dir) @@ -1961,6 +1957,13 @@ class ClickHouseCluster: ) ) + ### !!!! This is the last step after combining all cmds, don't put anything after + if self.with_net_trics: + for cmd in cmds: + # Again, adding it only once + if docker_compose_net not in cmd: + cmd.extend(["--file", docker_compose_net]) + logging.debug( "Cluster name:{} project_name:{}. Added instance name:{} tag:{} base_cmd:{} docker_compose_yml_dir:{}".format( self.name, @@ -3243,8 +3246,7 @@ class ClickHouseCluster: subprocess_check_call(self.base_zookeeper_cmd + ["start", n]) -DOCKER_COMPOSE_TEMPLATE = """ -version: '2.3' +DOCKER_COMPOSE_TEMPLATE = """--- services: {name}: image: {image}:{tag} From f6fbd9c8aabaa7eaa67aaf0c9cf0609bf8c119d0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Oct 2024 15:32:19 +0200 Subject: [PATCH 59/65] Fix test_zookeeper_config_load_balancing after docker-compose upgrade --- .../test.py | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_zookeeper_config_load_balancing/test.py b/tests/integration/test_zookeeper_config_load_balancing/test.py index 0fa4a90d51a..21c3b677f81 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test.py @@ -73,7 +73,7 @@ def test_first_or_random(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -101,7 +101,7 @@ def test_first_or_random(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -129,7 +129,7 @@ def test_first_or_random(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -163,7 +163,7 @@ def test_in_order(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -191,7 +191,7 @@ def test_in_order(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -219,7 +219,7 @@ def test_in_order(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -253,7 +253,7 @@ def test_nearest_hostname(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -281,7 +281,7 @@ def test_nearest_hostname(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo2_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo2-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -309,7 +309,7 @@ def test_nearest_hostname(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo3_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo3-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -343,7 +343,7 @@ def test_hostname_levenshtein_distance(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo1_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -371,7 +371,7 @@ def test_hostname_levenshtein_distance(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo2_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo2-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", @@ -399,7 +399,7 @@ def test_hostname_levenshtein_distance(started_cluster): [ "bash", "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing_(gw\\d+_)?zoo3_1.*testzookeeperconfigloadbalancing_(gw\\d+_)?default:2181' | grep ESTABLISHED | wc -l", + "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo3-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", ], privileged=True, user="root", From 4de6e20f720c10bdf083638104ac8a5bd8f19a48 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Oct 2024 15:46:47 +0200 Subject: [PATCH 60/65] Remove `version` from the rest of docker-compose files --- tests/integration/compose/docker_compose_azurite.yml | 2 -- tests/integration/compose/docker_compose_cassandra.yml | 1 - tests/integration/compose/docker_compose_clickhouse.yml | 1 - tests/integration/compose/docker_compose_coredns.yml | 2 -- tests/integration/compose/docker_compose_dotnet_client.yml | 1 - tests/integration/compose/docker_compose_hdfs.yml | 1 - tests/integration/compose/docker_compose_hive.yml | 1 - tests/integration/compose/docker_compose_jdbc_bridge.yml | 3 +-- tests/integration/compose/docker_compose_kafka.yml | 2 -- tests/integration/compose/docker_compose_keeper.yml | 1 - tests/integration/compose/docker_compose_kerberized_hdfs.yml | 2 -- tests/integration/compose/docker_compose_kerberized_kafka.yml | 2 -- tests/integration/compose/docker_compose_kerberos_kdc.yml | 2 -- tests/integration/compose/docker_compose_ldap.yml | 1 - tests/integration/compose/docker_compose_minio.yml | 2 -- tests/integration/compose/docker_compose_mongo.yml | 1 - tests/integration/compose/docker_compose_mysql.yml | 1 - tests/integration/compose/docker_compose_mysql_8_0.yml | 1 - tests/integration/compose/docker_compose_mysql_client.yml | 1 - tests/integration/compose/docker_compose_mysql_cluster.yml | 1 - .../integration/compose/docker_compose_mysql_golang_client.yml | 1 - tests/integration/compose/docker_compose_mysql_java_client.yml | 1 - tests/integration/compose/docker_compose_mysql_js_client.yml | 1 - tests/integration/compose/docker_compose_mysql_php_client.yml | 1 - tests/integration/compose/docker_compose_nats.yml | 1 - tests/integration/compose/docker_compose_net.yml | 1 - tests/integration/compose/docker_compose_nginx.yml | 1 - tests/integration/compose/docker_compose_postgres.yml | 1 - tests/integration/compose/docker_compose_postgres_cluster.yml | 3 +-- tests/integration/compose/docker_compose_postgresql.yml | 1 - .../compose/docker_compose_postgresql_java_client.yml | 1 - tests/integration/compose/docker_compose_prometheus.yml | 1 - tests/integration/compose/docker_compose_rabbitmq.yml | 2 -- tests/integration/compose/docker_compose_redis.yml | 1 - tests/integration/compose/docker_compose_zookeeper.yml | 1 - tests/integration/compose/docker_compose_zookeeper_secure.yml | 1 - 36 files changed, 2 insertions(+), 46 deletions(-) diff --git a/tests/integration/compose/docker_compose_azurite.yml b/tests/integration/compose/docker_compose_azurite.yml index 7c379a971ea..8ae9f7b85ff 100644 --- a/tests/integration/compose/docker_compose_azurite.yml +++ b/tests/integration/compose/docker_compose_azurite.yml @@ -1,5 +1,3 @@ -version: '2.3' - services: azurite1: image: mcr.microsoft.com/azure-storage/azurite diff --git a/tests/integration/compose/docker_compose_cassandra.yml b/tests/integration/compose/docker_compose_cassandra.yml index b6190a11d73..98bc5a22fb3 100644 --- a/tests/integration/compose/docker_compose_cassandra.yml +++ b/tests/integration/compose/docker_compose_cassandra.yml @@ -1,4 +1,3 @@ -version: '2.3' services: cassandra1: image: cassandra:4.0 diff --git a/tests/integration/compose/docker_compose_clickhouse.yml b/tests/integration/compose/docker_compose_clickhouse.yml index 9702649f1b8..8b73db02903 100644 --- a/tests/integration/compose/docker_compose_clickhouse.yml +++ b/tests/integration/compose/docker_compose_clickhouse.yml @@ -1,4 +1,3 @@ -version: '2.3' # Used to pre-pull images with docker compose services: clickhouse1: diff --git a/tests/integration/compose/docker_compose_coredns.yml b/tests/integration/compose/docker_compose_coredns.yml index e4736e04846..90b714f5c2c 100644 --- a/tests/integration/compose/docker_compose_coredns.yml +++ b/tests/integration/compose/docker_compose_coredns.yml @@ -1,5 +1,3 @@ -version: "2.3" - services: coredns: image: coredns/coredns:1.9.3 # :latest broke this test diff --git a/tests/integration/compose/docker_compose_dotnet_client.yml b/tests/integration/compose/docker_compose_dotnet_client.yml index b63dac51522..b44a47da5b1 100644 --- a/tests/integration/compose/docker_compose_dotnet_client.yml +++ b/tests/integration/compose/docker_compose_dotnet_client.yml @@ -1,4 +1,3 @@ -version: '2.3' services: dotnet1: image: clickhouse/dotnet-client:${DOCKER_DOTNET_CLIENT_TAG:-latest} diff --git a/tests/integration/compose/docker_compose_hdfs.yml b/tests/integration/compose/docker_compose_hdfs.yml index 40a10df01f7..1635219e333 100644 --- a/tests/integration/compose/docker_compose_hdfs.yml +++ b/tests/integration/compose/docker_compose_hdfs.yml @@ -1,4 +1,3 @@ -version: '2.3' services: hdfs1: image: prasanthj/docker-hadoop:2.6.0 diff --git a/tests/integration/compose/docker_compose_hive.yml b/tests/integration/compose/docker_compose_hive.yml index 459e8481d0b..16253e50f4c 100644 --- a/tests/integration/compose/docker_compose_hive.yml +++ b/tests/integration/compose/docker_compose_hive.yml @@ -1,4 +1,3 @@ -version: '2.3' services: hdfs1: image: lgboustc/hive_test:v2.0 diff --git a/tests/integration/compose/docker_compose_jdbc_bridge.yml b/tests/integration/compose/docker_compose_jdbc_bridge.yml index b3686adc21c..26f575923a2 100644 --- a/tests/integration/compose/docker_compose_jdbc_bridge.yml +++ b/tests/integration/compose/docker_compose_jdbc_bridge.yml @@ -1,4 +1,3 @@ -version: '2.3' services: bridge1: image: clickhouse/jdbc-bridge @@ -24,4 +23,4 @@ services: volumes: - type: ${JDBC_BRIDGE_FS:-tmpfs} source: ${JDBC_BRIDGE_LOGS:-} - target: /app/logs \ No newline at end of file + target: /app/logs diff --git a/tests/integration/compose/docker_compose_kafka.yml b/tests/integration/compose/docker_compose_kafka.yml index 4ae3de3cbc7..e4ee9fbc0b9 100644 --- a/tests/integration/compose/docker_compose_kafka.yml +++ b/tests/integration/compose/docker_compose_kafka.yml @@ -1,5 +1,3 @@ -version: '2.3' - services: kafka_zookeeper: image: zookeeper:3.4.9 diff --git a/tests/integration/compose/docker_compose_keeper.yml b/tests/integration/compose/docker_compose_keeper.yml index 91010c4aa83..4b2fe1e637a 100644 --- a/tests/integration/compose/docker_compose_keeper.yml +++ b/tests/integration/compose/docker_compose_keeper.yml @@ -1,4 +1,3 @@ -version: '2.3' services: zoo1: image: ${image:-clickhouse/integration-test} diff --git a/tests/integration/compose/docker_compose_kerberized_hdfs.yml b/tests/integration/compose/docker_compose_kerberized_hdfs.yml index e955a14eb3d..4354f4aba01 100644 --- a/tests/integration/compose/docker_compose_kerberized_hdfs.yml +++ b/tests/integration/compose/docker_compose_kerberized_hdfs.yml @@ -1,5 +1,3 @@ -version: '2.3' - services: kerberizedhdfs1: cap_add: diff --git a/tests/integration/compose/docker_compose_kerberized_kafka.yml b/tests/integration/compose/docker_compose_kerberized_kafka.yml index 49d4c1db90f..90bcf11a50f 100644 --- a/tests/integration/compose/docker_compose_kerberized_kafka.yml +++ b/tests/integration/compose/docker_compose_kerberized_kafka.yml @@ -1,5 +1,3 @@ -version: '2.3' - services: kafka_kerberized_zookeeper: image: confluentinc/cp-zookeeper:5.2.0 diff --git a/tests/integration/compose/docker_compose_kerberos_kdc.yml b/tests/integration/compose/docker_compose_kerberos_kdc.yml index 3ce9a6df1fb..8cdac4118b9 100644 --- a/tests/integration/compose/docker_compose_kerberos_kdc.yml +++ b/tests/integration/compose/docker_compose_kerberos_kdc.yml @@ -1,5 +1,3 @@ -version: '2.3' - services: kerberoskdc: image: clickhouse/kerberos-kdc:${DOCKER_KERBEROS_KDC_TAG:-latest} diff --git a/tests/integration/compose/docker_compose_ldap.yml b/tests/integration/compose/docker_compose_ldap.yml index f49e00400a2..440a271272b 100644 --- a/tests/integration/compose/docker_compose_ldap.yml +++ b/tests/integration/compose/docker_compose_ldap.yml @@ -1,4 +1,3 @@ -version: '2.3' services: openldap: image: bitnami/openldap:2.6.6 diff --git a/tests/integration/compose/docker_compose_minio.yml b/tests/integration/compose/docker_compose_minio.yml index 40098d05b04..44a07e97843 100644 --- a/tests/integration/compose/docker_compose_minio.yml +++ b/tests/integration/compose/docker_compose_minio.yml @@ -1,5 +1,3 @@ -version: '2.3' - services: minio1: image: minio/minio:RELEASE.2024-07-31T05-46-26Z diff --git a/tests/integration/compose/docker_compose_mongo.yml b/tests/integration/compose/docker_compose_mongo.yml index fbbfac17aeb..774190f4d27 100644 --- a/tests/integration/compose/docker_compose_mongo.yml +++ b/tests/integration/compose/docker_compose_mongo.yml @@ -1,4 +1,3 @@ -version: '2.3' services: mongo1: image: mongo:6.0 diff --git a/tests/integration/compose/docker_compose_mysql.yml b/tests/integration/compose/docker_compose_mysql.yml index 69f7f02fb4d..f45410bde78 100644 --- a/tests/integration/compose/docker_compose_mysql.yml +++ b/tests/integration/compose/docker_compose_mysql.yml @@ -1,4 +1,3 @@ -version: '2.3' services: mysql57: image: mysql:5.7 diff --git a/tests/integration/compose/docker_compose_mysql_8_0.yml b/tests/integration/compose/docker_compose_mysql_8_0.yml index 1e0ded6c6bd..e1ff1633bc7 100644 --- a/tests/integration/compose/docker_compose_mysql_8_0.yml +++ b/tests/integration/compose/docker_compose_mysql_8_0.yml @@ -1,4 +1,3 @@ -version: '2.3' services: mysql80: image: mysql:8.0 diff --git a/tests/integration/compose/docker_compose_mysql_client.yml b/tests/integration/compose/docker_compose_mysql_client.yml index ee590118d4f..74262d61d9a 100644 --- a/tests/integration/compose/docker_compose_mysql_client.yml +++ b/tests/integration/compose/docker_compose_mysql_client.yml @@ -1,4 +1,3 @@ -version: '2.3' services: mysql_client: image: mysql:8.0 diff --git a/tests/integration/compose/docker_compose_mysql_cluster.yml b/tests/integration/compose/docker_compose_mysql_cluster.yml index 3f7d21b733f..e065cea9d5d 100644 --- a/tests/integration/compose/docker_compose_mysql_cluster.yml +++ b/tests/integration/compose/docker_compose_mysql_cluster.yml @@ -1,4 +1,3 @@ -version: '2.3' services: mysql2: image: mysql:8.0 diff --git a/tests/integration/compose/docker_compose_mysql_golang_client.yml b/tests/integration/compose/docker_compose_mysql_golang_client.yml index 56cc0410574..5268978b0fe 100644 --- a/tests/integration/compose/docker_compose_mysql_golang_client.yml +++ b/tests/integration/compose/docker_compose_mysql_golang_client.yml @@ -1,4 +1,3 @@ -version: '2.3' services: golang1: image: clickhouse/mysql-golang-client:${DOCKER_MYSQL_GOLANG_CLIENT_TAG:-latest} diff --git a/tests/integration/compose/docker_compose_mysql_java_client.yml b/tests/integration/compose/docker_compose_mysql_java_client.yml index 529974dd4bf..20c95a7d51e 100644 --- a/tests/integration/compose/docker_compose_mysql_java_client.yml +++ b/tests/integration/compose/docker_compose_mysql_java_client.yml @@ -1,4 +1,3 @@ -version: '2.3' services: java1: image: clickhouse/mysql-java-client:${DOCKER_MYSQL_JAVA_CLIENT_TAG:-latest} diff --git a/tests/integration/compose/docker_compose_mysql_js_client.yml b/tests/integration/compose/docker_compose_mysql_js_client.yml index 90939449c5f..be4edaead4a 100644 --- a/tests/integration/compose/docker_compose_mysql_js_client.yml +++ b/tests/integration/compose/docker_compose_mysql_js_client.yml @@ -1,4 +1,3 @@ -version: '2.3' services: mysqljs1: image: clickhouse/mysql-js-client:${DOCKER_MYSQL_JS_CLIENT_TAG:-latest} diff --git a/tests/integration/compose/docker_compose_mysql_php_client.yml b/tests/integration/compose/docker_compose_mysql_php_client.yml index 408b8ff089a..0b00dedf152 100644 --- a/tests/integration/compose/docker_compose_mysql_php_client.yml +++ b/tests/integration/compose/docker_compose_mysql_php_client.yml @@ -1,4 +1,3 @@ -version: '2.3' services: php1: image: clickhouse/mysql-php-client:${DOCKER_MYSQL_PHP_CLIENT_TAG:-latest} diff --git a/tests/integration/compose/docker_compose_nats.yml b/tests/integration/compose/docker_compose_nats.yml index b17ac62fa93..059b538218b 100644 --- a/tests/integration/compose/docker_compose_nats.yml +++ b/tests/integration/compose/docker_compose_nats.yml @@ -1,4 +1,3 @@ -version: '2.3' services: nats1: image: nats diff --git a/tests/integration/compose/docker_compose_net.yml b/tests/integration/compose/docker_compose_net.yml index 311f3008639..d98f817071c 100644 --- a/tests/integration/compose/docker_compose_net.yml +++ b/tests/integration/compose/docker_compose_net.yml @@ -1,4 +1,3 @@ -version: '2.3' networks: default: driver: bridge diff --git a/tests/integration/compose/docker_compose_nginx.yml b/tests/integration/compose/docker_compose_nginx.yml index 38d2a6d84c8..2767a3c6f81 100644 --- a/tests/integration/compose/docker_compose_nginx.yml +++ b/tests/integration/compose/docker_compose_nginx.yml @@ -1,4 +1,3 @@ -version: '2.3' services: # nginx server to host static files. # Accepts only PUT data by test.com/path and GET already existing data on test.com/path. diff --git a/tests/integration/compose/docker_compose_postgres.yml b/tests/integration/compose/docker_compose_postgres.yml index c55cd8a31cf..cd1debce771 100644 --- a/tests/integration/compose/docker_compose_postgres.yml +++ b/tests/integration/compose/docker_compose_postgres.yml @@ -1,4 +1,3 @@ -version: '2.3' services: postgres1: image: postgres diff --git a/tests/integration/compose/docker_compose_postgres_cluster.yml b/tests/integration/compose/docker_compose_postgres_cluster.yml index 5af13ca3e0f..0458dee6320 100644 --- a/tests/integration/compose/docker_compose_postgres_cluster.yml +++ b/tests/integration/compose/docker_compose_postgres_cluster.yml @@ -1,4 +1,3 @@ -version: '2.3' services: postgres2: image: postgres @@ -41,4 +40,4 @@ services: volumes: - type: ${POSTGRES_LOGS_FS:-tmpfs} source: ${POSTGRES4_DIR:-} - target: /postgres/ \ No newline at end of file + target: /postgres/ diff --git a/tests/integration/compose/docker_compose_postgresql.yml b/tests/integration/compose/docker_compose_postgresql.yml index 90764188ddd..79c10bf175e 100644 --- a/tests/integration/compose/docker_compose_postgresql.yml +++ b/tests/integration/compose/docker_compose_postgresql.yml @@ -1,4 +1,3 @@ -version: '2.2' services: psql: image: postgres:12.2-alpine diff --git a/tests/integration/compose/docker_compose_postgresql_java_client.yml b/tests/integration/compose/docker_compose_postgresql_java_client.yml index 904bfffdfd5..133dccd569e 100644 --- a/tests/integration/compose/docker_compose_postgresql_java_client.yml +++ b/tests/integration/compose/docker_compose_postgresql_java_client.yml @@ -1,4 +1,3 @@ -version: '2.2' services: java: image: clickhouse/postgresql-java-client:${DOCKER_POSTGRESQL_JAVA_CLIENT_TAG:-latest} diff --git a/tests/integration/compose/docker_compose_prometheus.yml b/tests/integration/compose/docker_compose_prometheus.yml index 0a1db2138ba..24710b971a8 100644 --- a/tests/integration/compose/docker_compose_prometheus.yml +++ b/tests/integration/compose/docker_compose_prometheus.yml @@ -1,4 +1,3 @@ -version: '2.3' services: prometheus_writer: image: prom/prometheus:v2.50.1 diff --git a/tests/integration/compose/docker_compose_rabbitmq.yml b/tests/integration/compose/docker_compose_rabbitmq.yml index 94c7f0111c4..4aae2427596 100644 --- a/tests/integration/compose/docker_compose_rabbitmq.yml +++ b/tests/integration/compose/docker_compose_rabbitmq.yml @@ -1,5 +1,3 @@ -version: '2.3' - services: rabbitmq1: image: rabbitmq:3.12.6-alpine diff --git a/tests/integration/compose/docker_compose_redis.yml b/tests/integration/compose/docker_compose_redis.yml index e2aa836ae46..21f303669ed 100644 --- a/tests/integration/compose/docker_compose_redis.yml +++ b/tests/integration/compose/docker_compose_redis.yml @@ -1,4 +1,3 @@ -version: '2.3' services: redis1: image: redis diff --git a/tests/integration/compose/docker_compose_zookeeper.yml b/tests/integration/compose/docker_compose_zookeeper.yml index 1601d217a25..708d2379360 100644 --- a/tests/integration/compose/docker_compose_zookeeper.yml +++ b/tests/integration/compose/docker_compose_zookeeper.yml @@ -1,4 +1,3 @@ -version: '2.3' services: zoo1: image: zookeeper:3.6.2 diff --git a/tests/integration/compose/docker_compose_zookeeper_secure.yml b/tests/integration/compose/docker_compose_zookeeper_secure.yml index b5dbae423b2..40b22717942 100644 --- a/tests/integration/compose/docker_compose_zookeeper_secure.yml +++ b/tests/integration/compose/docker_compose_zookeeper_secure.yml @@ -1,4 +1,3 @@ -version: '2.3' services: zoo1: image: zookeeper:3.6.2 From 63cb7dfa5f61b97a5610a145dfa842c2583d8fa7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Oct 2024 21:47:21 +0200 Subject: [PATCH 61/65] Fix some issues in cluster.py, improve logging --- tests/integration/helpers/cluster.py | 28 ++++++++++++++++++---------- 1 file changed, 18 insertions(+), 10 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 8b94aec1972..47ca63db420 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -18,7 +18,7 @@ import traceback import urllib.parse from functools import cache from pathlib import Path -from typing import List, Tuple +from typing import List, Sequence, Tuple, Union import requests import urllib3 @@ -86,7 +86,7 @@ CLICKHOUSE_CI_MIN_TESTED_VERSION = "23.3" # to create docker-compose env file def _create_env_file(path, variables): - logging.debug(f"Env {variables} stored in {path}") + logging.debug("Env %s stored in %s", variables, path) with open(path, "w") as f: for var, value in list(variables.items()): f.write("=".join([var, value]) + "\n") @@ -94,7 +94,7 @@ def _create_env_file(path, variables): def run_and_check( - args, + args: Union[Sequence[str], str], env=None, shell=False, stdout=subprocess.PIPE, @@ -103,6 +103,15 @@ def run_and_check( nothrow=False, detach=False, ) -> str: + if shell: + if isinstance(args, str): + shell_args = args + else: + shell_args = next(a for a in args) + else: + shell_args = " ".join(args) + + logging.debug("Command:[%s]", shell_args) if detach: subprocess.Popen( args, @@ -113,7 +122,6 @@ def run_and_check( ) return "" - logging.debug(f"Command:{args}") res = subprocess.run( args, stdout=stdout, @@ -127,16 +135,16 @@ def run_and_check( err = res.stderr.decode("utf-8", "ignore") # check_call(...) from subprocess does not print stderr, so we do it manually for outline in out.splitlines(): - logging.debug(f"Stdout:{outline}") + logging.debug("Stdout:%s", outline) for errline in err.splitlines(): - logging.debug(f"Stderr:{errline}") + logging.debug("Stderr:%s", errline) if res.returncode != 0: - logging.debug(f"Exitcode:{res.returncode}") + logging.debug("Exitcode:%s", res.returncode) if env: - logging.debug(f"Env:{env}") + logging.debug("Env:%s", env) if not nothrow: raise Exception( - f"Command {args} return non-zero code {res.returncode}: {res.stderr.decode('utf-8')}" + f"Command [{shell_args}] return non-zero code {res.returncode}: {res.stderr.decode('utf-8')}" ) return out @@ -935,7 +943,7 @@ class ClickHouseCluster: logging.debug("Trying to prune unused volumes...") result = run_and_check(["docker volume ls | wc -l"], shell=True) - if int(result > 0): + if int(result) > 1: run_and_check(["docker", "volume", "prune", "-f"]) logging.debug(f"Volumes pruned: {result}") except: From dadbb0a8ac73eb3c8da396f0f17788b7a13ef353 Mon Sep 17 00:00:00 2001 From: Justin de Guzman Date: Tue, 1 Oct 2024 22:14:32 -0700 Subject: [PATCH 62/65] [Docs] Correct async_insert_use_adaptive_busy_timeout --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 982a8f1dbf2..4fbafccb48a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2045,7 +2045,7 @@ Possible values: - 0 - Disabled. - 1 - Enabled. -Default value: `0`. +Default value: `1`. ### async_insert_busy_timeout_min_ms {#async-insert-busy-timeout-min-ms} From e04ce8b6b5a2e8c8b4e6affb695cffbd5a0b71eb Mon Sep 17 00:00:00 2001 From: Peter Date: Wed, 2 Oct 2024 13:41:31 +0800 Subject: [PATCH 63/65] Improve currentDatabase function usage --- .../functions/other-functions.md | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index d17d05165e8..24489f2b7e0 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -500,6 +500,30 @@ Useful in table engine parameters of `CREATE TABLE` queries where you need to sp currentDatabase() ``` +**Arguments** + +None. + +**Returned value** + +- `value` returns the current database name. [String](../data-types/string.md). + +**Example** + +Query: + +```sql +SELECT currentDatabase() +``` + +Result: + +```response +┌─currentDatabase()─┐ +│ default │ +└───────────────────┘ +``` + ## currentUser {#currentUser} Returns the name of the current user. In case of a distributed query, the name of the user who initiated the query is returned. From 6755039750485dbb6cabec244f8148615f029896 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 2 Oct 2024 00:37:54 +0200 Subject: [PATCH 64/65] Simplify test_zookeeper_config_load_balancing, use ss --- .../test.py | 386 +++--------------- 1 file changed, 57 insertions(+), 329 deletions(-) diff --git a/tests/integration/test_zookeeper_config_load_balancing/test.py b/tests/integration/test_zookeeper_config_load_balancing/test.py index 21c3b677f81..d72cccff783 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test.py @@ -1,3 +1,5 @@ +import logging +import re import time import pytest @@ -20,11 +22,33 @@ node2 = cluster.add_instance( node3 = cluster.add_instance( "nod3", with_zookeeper=True, main_configs=["configs/zookeeper_load_balancing.xml"] ) - node4 = cluster.add_instance( "nod4", with_zookeeper=True, main_configs=["configs/zookeeper_load_balancing2.xml"] ) +ss_established = [ + "ss", + "--resolve", + "--tcp", + "--no-header", + "state", + "ESTABLISHED", + "( dport = 2181 or sport = 2181 )", +] + +zk1_re = re.compile( + r"testzookeeperconfigloadbalancing-(gw\d+-)?zoo1-1" + r".*testzookeeperconfigloadbalancing(-gw\d+)?_default:2181" +) +zk2_re = re.compile( + r"testzookeeperconfigloadbalancing-(gw\d+-)?zoo2-1" + r".*testzookeeperconfigloadbalancing(-gw\d+)?_default:2181" +) +zk3_re = re.compile( + r"testzookeeperconfigloadbalancing-(gw\d+-)?zoo3-1" + r".*testzookeeperconfigloadbalancing(-gw\d+)?_default:2181" +) + def change_balancing(old, new, reload=True): line = "{}<" @@ -53,89 +77,15 @@ def started_cluster(): def test_first_or_random(started_cluster): try: change_balancing("random", "first_or_random") - print( - str( - node1.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", - ], - privileged=True, - user="root", - ) + for node in (node1, node2, node3): + connections = ( + node.exec_in_container(ss_established, privileged=True, user="root") + .strip() + .split("\n") ) - ) - assert ( - "1" - == str( - node1.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - ) - - print( - str( - node2.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", - ], - privileged=True, - user="root", - ) - ) - ) - assert ( - "1" - == str( - node2.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - ) - - print( - str( - node3.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", - ], - privileged=True, - user="root", - ) - ) - ) - assert ( - "1" - == str( - node3.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - ) + logging.debug("Established connections for 2181:\n%s", connections) + assert len(connections) == 1 + assert zk1_re.search(connections[0]) finally: change_balancing("first_or_random", "random", reload=False) @@ -143,89 +93,15 @@ def test_first_or_random(started_cluster): def test_in_order(started_cluster): try: change_balancing("random", "in_order") - print( - str( - node1.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", - ], - privileged=True, - user="root", - ) + for node in (node1, node2, node3): + connections = ( + node.exec_in_container(ss_established, privileged=True, user="root") + .strip() + .split("\n") ) - ) - assert ( - "1" - == str( - node1.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - ) - - print( - str( - node2.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", - ], - privileged=True, - user="root", - ) - ) - ) - assert ( - "1" - == str( - node2.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - ) - - print( - str( - node3.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", - ], - privileged=True, - user="root", - ) - ) - ) - assert ( - "1" - == str( - node3.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - ) + logging.debug("Established connections for 2181:\n%s", connections) + assert len(connections) == 1 + assert zk1_re.search(connections[0]) finally: change_balancing("in_order", "random", reload=False) @@ -233,89 +109,15 @@ def test_in_order(started_cluster): def test_nearest_hostname(started_cluster): try: change_balancing("random", "nearest_hostname") - print( - str( - node1.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", - ], - privileged=True, - user="root", - ) + for node, regexp in ((node1, zk1_re), (node2, zk2_re), (node3, zk3_re)): + connections = ( + node.exec_in_container(ss_established, privileged=True, user="root") + .strip() + .split("\n") ) - ) - assert ( - "1" - == str( - node1.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - ) - - print( - str( - node2.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", - ], - privileged=True, - user="root", - ) - ) - ) - assert ( - "1" - == str( - node2.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo2-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - ) - - print( - str( - node3.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", - ], - privileged=True, - user="root", - ) - ) - ) - assert ( - "1" - == str( - node3.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo3-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - ) + logging.debug("Established connections for 2181:\n%s", connections) + assert len(connections) == 1 + assert regexp.search(connections[0]) finally: change_balancing("nearest_hostname", "random", reload=False) @@ -323,89 +125,15 @@ def test_nearest_hostname(started_cluster): def test_hostname_levenshtein_distance(started_cluster): try: change_balancing("random", "hostname_levenshtein_distance") - print( - str( - node1.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", - ], - privileged=True, - user="root", - ) + for node, regexp in ((node1, zk1_re), (node2, zk2_re), (node3, zk3_re)): + connections = ( + node.exec_in_container(ss_established, privileged=True, user="root") + .strip() + .split("\n") ) - ) - assert ( - "1" - == str( - node1.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo1-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - ) - - print( - str( - node2.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", - ], - privileged=True, - user="root", - ) - ) - ) - assert ( - "1" - == str( - node2.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo2-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - ) - - print( - str( - node3.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep ':2181' | grep ESTABLISHED", - ], - privileged=True, - user="root", - ) - ) - ) - assert ( - "1" - == str( - node3.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep -P 'testzookeeperconfigloadbalancing-(gw\\d+-)?zoo3-1.*testzookeeperconfigloadbalancing-(gw\\d+-)?default:2181' | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - ) + logging.debug("Established connections for 2181:\n%s", connections) + assert len(connections) == 1 + assert regexp.search(connections[0]) finally: change_balancing("hostname_levenshtein_distance", "random", reload=False) From 84c81cdc3cbdd7609d73b698266b8e0308824ea2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 2 Oct 2024 00:54:11 +0200 Subject: [PATCH 65/65] Move the common keeper code to helpers/keeper_utils.py --- tests/integration/helpers/keeper_utils.py | 32 +++++++++++++++++++ .../integration/test_read_only_table/test.py | 22 +------------ .../integration/test_reload_zookeeper/test.py | 22 +++---------- .../test.py | 23 +++---------- .../integration/test_replicated_users/test.py | 22 +++---------- .../test.py | 11 +------ 6 files changed, 49 insertions(+), 83 deletions(-) diff --git a/tests/integration/helpers/keeper_utils.py b/tests/integration/helpers/keeper_utils.py index bc08e7b120d..ae57c09825a 100644 --- a/tests/integration/helpers/keeper_utils.py +++ b/tests/integration/helpers/keeper_utils.py @@ -1,5 +1,6 @@ import contextlib import io +import re import select import socket import subprocess @@ -11,6 +12,37 @@ from kazoo.client import KazooClient from helpers.client import CommandRequest from helpers.cluster import ClickHouseCluster, ClickHouseInstance +ss_established = [ + "ss", + "--resolve", + "--tcp", + "--no-header", + "state", + "ESTABLISHED", + "( dport = 2181 or sport = 2181 )", +] + + +def get_active_zk_connections(node: ClickHouseInstance) -> tp.List[str]: + return ( + str(node.exec_in_container(ss_established, privileged=True, user="root")) + .strip() + .split("\n") + ) + + +def get_zookeeper_which_node_connected_to(node: ClickHouseInstance) -> str: + line = str( + node.exec_in_container(ss_established, privileged=True, user="root") + ).strip() + + pattern = re.compile(r"zoo[0-9]+", re.IGNORECASE) + result = pattern.findall(line) + assert ( + len(result) == 1 + ), "ClickHouse must be connected only to one Zookeeper at a time" + return result[0] + def execute_keeper_client_query( cluster: ClickHouseCluster, node: ClickHouseInstance, query: str diff --git a/tests/integration/test_read_only_table/test.py b/tests/integration/test_read_only_table/test.py index e44f3b49d33..cf5f82a5b1e 100644 --- a/tests/integration/test_read_only_table/test.py +++ b/tests/integration/test_read_only_table/test.py @@ -1,10 +1,10 @@ import logging -import re import time import pytest from helpers.cluster import ClickHouseCluster +from helpers.keeper_utils import get_zookeeper_which_node_connected_to from helpers.test_tools import assert_eq_with_retry NUM_TABLES = 10 @@ -57,26 +57,6 @@ def test_restart_zookeeper(start_cluster): logging.info("Inserted test data and initialized all tables") - def get_zookeeper_which_node_connected_to(node): - line = str( - node.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep 2181 | grep ESTABLISHED", - ], - privileged=True, - user="root", - ) - ).strip() - - pattern = re.compile(r"zoo[0-9]+", re.IGNORECASE) - result = pattern.findall(line) - assert ( - len(result) == 1 - ), "ClickHouse must be connected only to one Zookeeper at a time" - return result[0] - node1_zk = get_zookeeper_which_node_connected_to(node1) # ClickHouse should +- immediately reconnect to another zookeeper node diff --git a/tests/integration/test_reload_zookeeper/test.py b/tests/integration/test_reload_zookeeper/test.py index 3014c22b5f1..78e997da9e6 100644 --- a/tests/integration/test_reload_zookeeper/test.py +++ b/tests/integration/test_reload_zookeeper/test.py @@ -5,6 +5,7 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster +from helpers.keeper_utils import get_active_zk_connections from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__, zookeeper_config_path="configs/zookeeper.xml") @@ -85,19 +86,6 @@ def test_reload_zookeeper(start_cluster): settings={"select_sequential_consistency": 1}, ) - def get_active_zk_connections(): - return str( - node.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep 2181 | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - ## set config to zoo2, server will be normal new_config = """ @@ -113,16 +101,16 @@ def test_reload_zookeeper(start_cluster): node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) node.query("SYSTEM RELOAD CONFIG") - active_zk_connections = get_active_zk_connections() + active_zk_connections = get_active_zk_connections(node) assert ( - active_zk_connections == "1" + len(active_zk_connections) == 1 ), "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections) assert_eq_with_retry( node, "SELECT COUNT() FROM test_table", "1000", retry_count=120, sleep_time=0.5 ) - active_zk_connections = get_active_zk_connections() + active_zk_connections = get_active_zk_connections(node) assert ( - active_zk_connections == "1" + len(active_zk_connections) == 1 ), "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections) diff --git a/tests/integration/test_replicated_user_defined_functions/test.py b/tests/integration/test_replicated_user_defined_functions/test.py index ed64bece846..aba507a569c 100644 --- a/tests/integration/test_replicated_user_defined_functions/test.py +++ b/tests/integration/test_replicated_user_defined_functions/test.py @@ -7,6 +7,7 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster +from helpers.keeper_utils import get_active_zk_connections from helpers.test_tools import TSV, assert_eq_with_retry SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -65,20 +66,6 @@ def revert_zookeeper_config(): replace_zookeeper_config(f.read()) -def get_active_zk_connections(): - return str( - node1.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep 2181 | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - - def test_create_and_drop(): node1.query("CREATE FUNCTION f1 AS (x, y) -> x + y") assert node1.query("SELECT f1(12, 3)") == "15\n" @@ -266,9 +253,9 @@ def test_reload_zookeeper(): ) ) - active_zk_connections = get_active_zk_connections() + active_zk_connections = get_active_zk_connections(node1) assert ( - active_zk_connections == "1" + len(active_zk_connections) == 1 ), "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections) node1.query("CREATE FUNCTION f3 AS (x, y) -> x / y") @@ -280,9 +267,9 @@ def test_reload_zookeeper(): assert node2.query("SELECT f1(12, 3), f2(), f3(12, 3)") == TSV([[15, 2, 4]]) - active_zk_connections = get_active_zk_connections() + active_zk_connections = get_active_zk_connections(node1) assert ( - active_zk_connections == "1" + len(active_zk_connections) == 1 ), "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections) node1.query("DROP FUNCTION f1") diff --git a/tests/integration/test_replicated_users/test.py b/tests/integration/test_replicated_users/test.py index 2a998b31395..92d6b7b614e 100644 --- a/tests/integration/test_replicated_users/test.py +++ b/tests/integration/test_replicated_users/test.py @@ -5,6 +5,7 @@ from dataclasses import dataclass import pytest from helpers.cluster import ClickHouseCluster +from helpers.keeper_utils import get_active_zk_connections from helpers.test_tools import TSV, assert_eq_with_retry cluster = ClickHouseCluster(__file__, zookeeper_config_path="configs/zookeeper.xml") @@ -189,19 +190,6 @@ def test_reload_zookeeper(started_cluster): node1.query("SYSTEM RELOAD CONFIG") node2.query("SYSTEM RELOAD CONFIG") - def get_active_zk_connections(): - return str( - node1.exec_in_container( - [ - "bash", - "-c", - "lsof -a -i4 -i6 -itcp -w | grep 2181 | grep ESTABLISHED | wc -l", - ], - privileged=True, - user="root", - ) - ).strip() - node1.query("CREATE USER u1") assert_eq_with_retry( node2, "SELECT name FROM system.users WHERE name ='u1'", "u1\n" @@ -260,9 +248,9 @@ def test_reload_zookeeper(started_cluster): """ ) - active_zk_connections = get_active_zk_connections() + active_zk_connections = get_active_zk_connections(node1) assert ( - active_zk_connections == "1" + len(active_zk_connections) == 1 ), "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections) node1.query("CREATE USER u3") @@ -272,7 +260,7 @@ def test_reload_zookeeper(started_cluster): TSV(["u1", "u2", "u3"]), ) - active_zk_connections = get_active_zk_connections() + active_zk_connections = get_active_zk_connections(node1) assert ( - active_zk_connections == "1" + len(active_zk_connections) == 1 ), "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections) diff --git a/tests/integration/test_zookeeper_config_load_balancing/test.py b/tests/integration/test_zookeeper_config_load_balancing/test.py index d72cccff783..3284d75ea65 100644 --- a/tests/integration/test_zookeeper_config_load_balancing/test.py +++ b/tests/integration/test_zookeeper_config_load_balancing/test.py @@ -5,6 +5,7 @@ import time import pytest from helpers.cluster import ClickHouseCluster +from helpers.keeper_utils import ss_established from helpers.network import PartitionManager from helpers.test_tools import assert_eq_with_retry @@ -26,16 +27,6 @@ node4 = cluster.add_instance( "nod4", with_zookeeper=True, main_configs=["configs/zookeeper_load_balancing2.xml"] ) -ss_established = [ - "ss", - "--resolve", - "--tcp", - "--no-header", - "state", - "ESTABLISHED", - "( dport = 2181 or sport = 2181 )", -] - zk1_re = re.compile( r"testzookeeperconfigloadbalancing-(gw\d+-)?zoo1-1" r".*testzookeeperconfigloadbalancing(-gw\d+)?_default:2181"