From b503bad51d4f1068b17be967c4e5f9b6393d490e Mon Sep 17 00:00:00 2001 From: flynn Date: Sun, 28 Jul 2024 14:42:14 +0000 Subject: [PATCH 01/27] Fix crash of uniq and uniqTheta with empty tuple argument --- src/AggregateFunctions/UniqVariadicHash.h | 3 +++ tests/queries/0_stateless/03208_uniq_with_empty_tuple.sql | 3 +++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/03208_uniq_with_empty_tuple.sql diff --git a/src/AggregateFunctions/UniqVariadicHash.h b/src/AggregateFunctions/UniqVariadicHash.h index 5bb245397d4..279feed8bc6 100644 --- a/src/AggregateFunctions/UniqVariadicHash.h +++ b/src/AggregateFunctions/UniqVariadicHash.h @@ -67,6 +67,9 @@ struct UniqVariadicHash { static UInt64 apply(size_t num_args, const IColumn ** columns, size_t row_num) { + if (!num_args) + return 0; + UInt64 hash; const auto & tuple_columns = assert_cast(columns[0])->getColumns(); diff --git a/tests/queries/0_stateless/03208_uniq_with_empty_tuple.sql b/tests/queries/0_stateless/03208_uniq_with_empty_tuple.sql new file mode 100644 index 00000000000..4c5705261f0 --- /dev/null +++ b/tests/queries/0_stateless/03208_uniq_with_empty_tuple.sql @@ -0,0 +1,3 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/67303 +SELECT uniqTheta(tuple()); +SELECT uniq(tuple()); From 19ce50f959b65fa1c1fd82b32c5d59252ed72383 Mon Sep 17 00:00:00 2001 From: flynn Date: Mon, 29 Jul 2024 02:36:28 +0000 Subject: [PATCH 02/27] Add missing reference file --- tests/queries/0_stateless/03208_uniq_with_empty_tuple.reference | 2 ++ tests/queries/0_stateless/03208_uniq_with_empty_tuple.sql | 1 + 2 files changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/03208_uniq_with_empty_tuple.reference diff --git a/tests/queries/0_stateless/03208_uniq_with_empty_tuple.reference b/tests/queries/0_stateless/03208_uniq_with_empty_tuple.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/03208_uniq_with_empty_tuple.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/03208_uniq_with_empty_tuple.sql b/tests/queries/0_stateless/03208_uniq_with_empty_tuple.sql index 4c5705261f0..09eeaf6f370 100644 --- a/tests/queries/0_stateless/03208_uniq_with_empty_tuple.sql +++ b/tests/queries/0_stateless/03208_uniq_with_empty_tuple.sql @@ -1,3 +1,4 @@ +-- Tags: no-fasttest -- https://github.com/ClickHouse/ClickHouse/issues/67303 SELECT uniqTheta(tuple()); SELECT uniq(tuple()); From cb056cf3a5080cbff61f6efd070733ae2061d5b8 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Mon, 29 Jul 2024 12:05:44 -0700 Subject: [PATCH 03/27] Add camelCase aliases for percentRank() and denseRank() for percent_rank() and dense_rank() --- src/Processors/Transforms/WindowTransform.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 2b255c5120e..a1b46c8e36c 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2721,20 +2721,24 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) parameters); }, properties}, AggregateFunctionFactory::Case::Insensitive); - factory.registerFunction("dense_rank", {[](const std::string & name, + factory.registerFunction("denseRank", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); }, properties}, AggregateFunctionFactory::Case::Insensitive); - factory.registerFunction("percent_rank", {[](const std::string & name, + factory.registerAlias("dense_rank", "denseRank", AggregateFunctionFactory::Case::Sensitive); + + factory.registerFunction("percentRank", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); }, properties}, AggregateFunctionFactory::Case::Insensitive); + factory.registerAlias("percent_rank", "percentRank", AggregateFunctionFactory::Case::Sensitive); + factory.registerFunction("row_number", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { From 9811a2e71b825a55c376edfb38303c817493cd9e Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Sun, 28 Jul 2024 17:32:32 -0700 Subject: [PATCH 04/27] Add test 03213_denseRank_percentRank_alias --- ...3213_denseRank_percentRank_alias.reference | 45 ++++++++++++++ .../03213_denseRank_percentRank_alias.sql | 59 +++++++++++++++++++ 2 files changed, 104 insertions(+) create mode 100644 tests/queries/0_stateless/03213_denseRank_percentRank_alias.reference create mode 100644 tests/queries/0_stateless/03213_denseRank_percentRank_alias.sql diff --git a/tests/queries/0_stateless/03213_denseRank_percentRank_alias.reference b/tests/queries/0_stateless/03213_denseRank_percentRank_alias.reference new file mode 100644 index 00000000000..b49e179df68 --- /dev/null +++ b/tests/queries/0_stateless/03213_denseRank_percentRank_alias.reference @@ -0,0 +1,45 @@ +---- denseRank() ---- +0 0 0 1 1 1 1 +3 0 0 2 2 2 2 +1 0 1 3 3 3 3 +4 0 1 4 4 4 4 +2 0 2 5 5 5 5 +6 1 0 1 1 1 1 +9 1 0 2 2 2 2 +7 1 1 3 3 3 3 +5 1 2 4 4 4 4 +8 1 2 5 5 5 5 +12 2 0 1 1 1 1 +10 2 1 2 2 2 2 +13 2 1 3 3 3 3 +11 2 2 4 4 4 4 +14 2 2 5 5 5 5 +15 3 0 1 1 1 1 +18 3 0 2 2 2 2 +16 3 1 3 3 3 3 +19 3 1 4 4 4 4 +17 3 2 5 5 5 5 +21 4 0 1 1 1 1 +24 4 0 2 2 2 2 +22 4 1 3 3 3 3 +20 4 2 4 4 4 4 +23 4 2 5 5 5 5 +27 5 0 1 1 1 1 +25 5 1 2 2 2 2 +28 5 1 3 3 3 3 +26 5 2 4 4 4 4 +29 5 2 5 5 5 5 +30 6 0 1 1 1 1 +---- percentRank() ---- +Lenovo Thinkpad Laptop 700 1 0 +Sony VAIO Laptop 700 1 0 +Dell Vostro Laptop 800 3 0.6666666666666666 +HP Elite Laptop 1200 4 1 +Microsoft Lumia Smartphone 200 1 0 +HTC One Smartphone 400 2 0.3333333333333333 +Nexus Smartphone 500 3 0.6666666666666666 +iPhone Smartphone 900 4 1 +Kindle Fire Tablet 150 1 0 +Samsung Galaxy Tab Tablet 200 2 0.5 +iPad Tablet 700 3 1 +Others Unknow 200 1 0 diff --git a/tests/queries/0_stateless/03213_denseRank_percentRank_alias.sql b/tests/queries/0_stateless/03213_denseRank_percentRank_alias.sql new file mode 100644 index 00000000000..ff841294eb1 --- /dev/null +++ b/tests/queries/0_stateless/03213_denseRank_percentRank_alias.sql @@ -0,0 +1,59 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/67042 +-- Reference generated using percent_rank() and dense_rank() + +-- From ClickHouse/tests/queries/0_stateless/01591_window_functions.sql (for deterministic query) +SELECT '---- denseRank() ----'; +select number, p, o, + count(*) over w, + rank() over w, + denseRank() over w, + row_number() over w +from (select number, intDiv(number, 5) p, mod(number, 3) o + from numbers(31) order by o, number) t +window w as (partition by p order by o, number) +order by p, o, number +settings max_block_size = 2; + +-- Modifed from ClickHouse/tests/queries/0_stateless/01592_window_functions.sql (for deterministic query) +SELECT '---- percentRank() ----'; + +drop table if exists product_groups; +drop table if exists products; + +CREATE TABLE product_groups ( + group_id Int64, + group_name String +) Engine = Memory; + +CREATE TABLE products ( + product_id Int64, + product_name String, + price DECIMAL(11, 2), + group_id Int64 +) Engine = Memory; + +INSERT INTO product_groups VALUES (1, 'Smartphone'),(2, 'Laptop'),(3, 'Tablet'); +INSERT INTO products (product_id,product_name, group_id,price) VALUES (1, 'Microsoft Lumia', 1, 200), (2, 'HTC One', 1, 400), (3, 'Nexus', 1, 500), (4, 'iPhone', 1, 900),(5, 'HP Elite', 2, 1200),(6, 'Lenovo Thinkpad', 2, 700),(7, 'Sony VAIO', 2, 700),(8, 'Dell Vostro', 2, 800),(9, 'iPad', 3, 700),(10, 'Kindle Fire', 3, 150),(11, 'Samsung Galaxy Tab', 3, 200); +INSERT INTO product_groups VALUES (4, 'Unknow'); +INSERT INTO products (product_id,product_name, group_id,price) VALUES (12, 'Others', 4, 200); + + +SELECT * +FROM +( + SELECT + product_name, + group_name, + price, + rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS rank, + percentRank() OVER (PARTITION BY group_name ORDER BY price ASC) AS percent + FROM products + INNER JOIN product_groups USING (group_id) +) AS t +ORDER BY + group_name ASC, + price ASC, + product_name ASC; + +drop table product_groups; +drop table products; From 4a4bd97b4b63e495f76273ea8c12045dc129d81b Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Tue, 30 Jul 2024 16:50:37 -0600 Subject: [PATCH 05/27] Fix case sensitivity for percent_rank, dense_rank, and their aliases --- src/Processors/Transforms/WindowTransform.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index a1b46c8e36c..5fad68e4968 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2726,18 +2726,18 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) { return std::make_shared(name, argument_types, parameters); - }, properties}, AggregateFunctionFactory::Case::Insensitive); + }, properties}); - factory.registerAlias("dense_rank", "denseRank", AggregateFunctionFactory::Case::Sensitive); + factory.registerAlias("dense_rank", "denseRank", AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("percentRank", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) { return std::make_shared(name, argument_types, parameters); - }, properties}, AggregateFunctionFactory::Case::Insensitive); + }, properties}); - factory.registerAlias("percent_rank", "percentRank", AggregateFunctionFactory::Case::Sensitive); + factory.registerAlias("percent_rank", "percentRank", AggregateFunctionFactory::Case::Insensitive); factory.registerFunction("row_number", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) From 3b12fec141fda8cb2a3ef68ac96e6e58f1fd69e3 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Tue, 30 Jul 2024 17:05:48 -0600 Subject: [PATCH 06/27] Update dense_rank doc to mention the denseRank alias --- docs/en/sql-reference/window-functions/dense_rank.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/window-functions/dense_rank.md b/docs/en/sql-reference/window-functions/dense_rank.md index d6445b68c55..2c8617fb668 100644 --- a/docs/en/sql-reference/window-functions/dense_rank.md +++ b/docs/en/sql-reference/window-functions/dense_rank.md @@ -12,6 +12,8 @@ The [rank](./rank.md) function provides the same behaviour, but with gaps in ran **Syntax** +Alias: `denseRank` (case-sensitive) + ```sql dense_rank (column_name) OVER ([[PARTITION BY grouping_column] [ORDER BY sorting_column] From 419a5e7f730dabe514becabc6c24ec5b87325e28 Mon Sep 17 00:00:00 2001 From: Peter Nguyen Date: Tue, 30 Jul 2024 17:17:01 -0600 Subject: [PATCH 07/27] Update window-functions doc with denseRank and percentRank aliases --- docs/en/sql-reference/window-functions/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 0c3e2ea1cb6..27d4bd763c7 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -23,8 +23,8 @@ ClickHouse supports the standard grammar for defining windows and window functio | `INTERVAL` syntax for `DateTime` `RANGE OFFSET` frame | ❌ (specify the number of seconds instead (`RANGE` works with any numeric type).) | | `GROUPS` frame | ❌ | | Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) | -| `rank()`, `dense_rank()`, `row_number()` | ✅ | -| `percent_rank()` | ✅ Efficiently computes the relative standing of a value within a partition in a dataset. This function effectively replaces the more verbose and computationally intensive manual SQL calculation expressed as `ifNull((rank() OVER(PARTITION BY x ORDER BY y) - 1) / nullif(count(1) OVER(PARTITION BY x) - 1, 0), 0)`| +| `rank()`, `dense_rank()`, `row_number()` | ✅
Alias: `denseRank()` | +| `percent_rank()` | ✅ Efficiently computes the relative standing of a value within a partition in a dataset. This function effectively replaces the more verbose and computationally intensive manual SQL calculation expressed as `ifNull((rank() OVER(PARTITION BY x ORDER BY y) - 1) / nullif(count(1) OVER(PARTITION BY x) - 1, 0), 0)`
Alias: `percentRank()`| | `lag/lead(value, offset)` | ❌
You can use one of the following workarounds:
1) `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead`
2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | | ntile(buckets) | ✅
Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | From d5f8540fbbcc7e5f8014212d67804412c7466d7d Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 12:00:09 +0000 Subject: [PATCH 08/27] Fix --- tests/integration/helpers/cluster.py | 25 +++++++++++++------- tests/integration/helpers/retry_decorator.py | 7 ++++-- 2 files changed, 21 insertions(+), 11 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7d80fbe90f8..7f0a9154be9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2692,11 +2692,13 @@ class ClickHouseCluster: images_pull_cmd = self.base_cmd + ["pull"] # sometimes dockerhub/proxy can be flaky - retry( - log_function=lambda exception: logging.info( - "Got exception pulling images: %s", exception - ), - )(run_and_check)(images_pull_cmd) + def logging_pulling_images(**kwargs): + if "exception" in kwargs: + logging.info( + "Got exception pulling images: %s", kwargs["exception"] + ) + + 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") @@ -2969,11 +2971,16 @@ class ClickHouseCluster: "Trying to create Azurite instance by command %s", " ".join(map(str, azurite_start_cmd)), ) - retry( - log_function=lambda exception: logging.info( + def logging_azurite_initialization(exception, retry_number, sleep_time): + logging.info( f"Azurite initialization failed with error: {exception}" - ), - )(run_and_check)(azurite_start_cmd) + ) + + retry( + log_function=logging_azurite_initialization, + )( + 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 aaa040464c2..e7bafbe29c1 100644 --- a/tests/integration/helpers/retry_decorator.py +++ b/tests/integration/helpers/retry_decorator.py @@ -8,7 +8,7 @@ def retry( delay: float = 1, backoff: float = 1.5, jitter: float = 2, - log_function=lambda *args, **kwargs: None, + log_function=None, # should take **kwargs or arguments: `retry_number`, `exception` and `sleep_time` retriable_expections_list: List[Type[BaseException]] = [Exception], ): def inner(func): @@ -26,8 +26,11 @@ def retry( break if not should_retry or (retry == retries - 1): raise e - log_function(retry=retry, exception=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 From 2a2dba63cc0182247754a5a4819cb89f21825bfd Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 31 Jul 2024 13:48:04 +0000 Subject: [PATCH 09/27] Automatic style fix --- tests/integration/helpers/cluster.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7f0a9154be9..2e38aec3512 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2971,6 +2971,7 @@ class ClickHouseCluster: "Trying to create Azurite instance by command %s", " ".join(map(str, azurite_start_cmd)), ) + def logging_azurite_initialization(exception, retry_number, sleep_time): logging.info( f"Azurite initialization failed with error: {exception}" From 67f4792b77f2a2cf0de21ead6e95c3635d26aa88 Mon Sep 17 00:00:00 2001 From: divanik Date: Wed, 31 Jul 2024 14:16:40 +0000 Subject: [PATCH 10/27] Style check --- tests/integration/helpers/cluster.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7f0a9154be9..2e38aec3512 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2971,6 +2971,7 @@ class ClickHouseCluster: "Trying to create Azurite instance by command %s", " ".join(map(str, azurite_start_cmd)), ) + def logging_azurite_initialization(exception, retry_number, sleep_time): logging.info( f"Azurite initialization failed with error: {exception}" From dde274f6fad979aa94ea31395b0434c81f72328a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 31 Jul 2024 18:08:14 +0000 Subject: [PATCH 11/27] Re-enable ICU on s390/x --- contrib/icu-cmake/CMakeLists.txt | 4 +--- contrib/icudata | 2 +- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/contrib/icu-cmake/CMakeLists.txt b/contrib/icu-cmake/CMakeLists.txt index f9d05f7fe97..adeaa7dcf33 100644 --- a/contrib/icu-cmake/CMakeLists.txt +++ b/contrib/icu-cmake/CMakeLists.txt @@ -4,9 +4,7 @@ else () option(ENABLE_ICU "Enable ICU" 0) endif () -# Temporarily disabled s390x because the ICU build links a blob (icudt71b_dat.S) and our friends from IBM did not explain how they generated -# the blob on s390x: https://github.com/ClickHouse/icudata/pull/2#issuecomment-2226957255 -if (NOT ENABLE_ICU OR ARCH_S390X) +if (NOT ENABLE_ICU) message(STATUS "Not using ICU") return() endif() diff --git a/contrib/icudata b/contrib/icudata index d345d6ac22f..4904951339a 160000 --- a/contrib/icudata +++ b/contrib/icudata @@ -1 +1 @@ -Subproject commit d345d6ac22f381c882420de9053d30ae1ff38d75 +Subproject commit 4904951339a70b4814d2d3723436b20d079cb01b From d683fb05a009ed3f58c0e11fc329c3783f934369 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Aug 2024 19:05:30 +0200 Subject: [PATCH 12/27] Fix --- .../IO/CachedOnDiskReadBufferFromFile.cpp | 17 ++++++--- src/Interpreters/Cache/FileCache.cpp | 35 ++++++++++++------- src/Interpreters/Cache/FileCache.h | 4 ++- src/Interpreters/Cache/FileSegment.cpp | 9 ++++- src/Interpreters/Cache/FileSegment.h | 4 ++- tests/config/config.d/storage_conf.xml | 3 +- 6 files changed, 51 insertions(+), 21 deletions(-) diff --git a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp index c928d25c7b8..b471f3fc58f 100644 --- a/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp +++ b/src/Disks/IO/CachedOnDiskReadBufferFromFile.cpp @@ -135,8 +135,11 @@ bool CachedOnDiskReadBufferFromFile::nextFileSegmentsBatch() else { CreateFileSegmentSettings create_settings(FileSegmentKind::Regular); - file_segments = cache->getOrSet(cache_key, file_offset_of_buffer_end, size, file_size.value(), create_settings, settings.filesystem_cache_segments_batch_size, user); + file_segments = cache->getOrSet( + cache_key, file_offset_of_buffer_end, size, file_size.value(), + create_settings, settings.filesystem_cache_segments_batch_size, user); } + return !file_segments->empty(); } @@ -158,8 +161,8 @@ void CachedOnDiskReadBufferFromFile::initialize() LOG_TEST( log, - "Having {} file segments to read: {}, current offset: {}", - file_segments->size(), file_segments->toString(), file_offset_of_buffer_end); + "Having {} file segments to read: {}, current read range: [{}, {})", + file_segments->size(), file_segments->toString(), file_offset_of_buffer_end, read_until_position); initialized = true; } @@ -1043,6 +1046,10 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() if (file_segments->size() == 1) { size_t remaining_size_to_read = std::min(current_read_range.right, read_until_position - 1) - file_offset_of_buffer_end + 1; + + LOG_TEST(log, "Remaining size to read: {}, read: {}. Resizing buffer to {}", + remaining_size_to_read, size, nextimpl_working_buffer_offset + std::min(size, remaining_size_to_read)); + size = std::min(size, remaining_size_to_read); chassert(implementation_buffer->buffer().size() >= nextimpl_working_buffer_offset + size); implementation_buffer->buffer().resize(nextimpl_working_buffer_offset + size); @@ -1055,8 +1062,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep() chassert( file_offset_of_buffer_end <= read_until_position, - fmt::format("Expected {} <= {} (size: {}, read range: {})", - file_offset_of_buffer_end, read_until_position, size, current_read_range.toString())); + fmt::format("Expected {} <= {} (size: {}, read range: {}, hold file segments: {} ({}))", + file_offset_of_buffer_end, read_until_position, size, current_read_range.toString(), file_segments->size(), file_segments->toString(true))); } swap(*implementation_buffer); diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index a88c0de2cfe..0a03f5dcc7d 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -316,14 +316,14 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: return result; } -std::vector FileCache::splitRange(size_t offset, size_t size) +std::vector FileCache::splitRange(size_t offset, size_t size, size_t aligned_size) { assert(size > 0); std::vector ranges; size_t current_pos = offset; size_t end_pos_non_included = offset + size; - size_t remaining_size = size; + size_t remaining_size = aligned_size; FileSegments file_segments; const size_t max_size = max_file_segment_size.load(); @@ -343,17 +343,20 @@ FileSegments FileCache::splitRangeIntoFileSegments( LockedKey & locked_key, size_t offset, size_t size, + size_t aligned_size, FileSegment::State state, size_t file_segments_limit, const CreateFileSegmentSettings & create_settings) { - assert(size > 0); + chassert(size > 0); + chassert(size <= aligned_size); + /// We take `size` as a soft limit and `aligned_size` as a hard limit. auto current_pos = offset; auto end_pos_non_included = offset + size; size_t current_file_segment_size; - size_t remaining_size = size; + size_t remaining_size = aligned_size; FileSegments file_segments; const size_t max_size = max_file_segment_size.load(); @@ -369,6 +372,8 @@ FileSegments FileCache::splitRangeIntoFileSegments( current_pos += current_file_segment_size; } + chassert(file_segments.size() == file_segments_limit || file_segments.back()->range().contains(offset + size - 1), + fmt::format("Offset: {}, size: {}, file segments: {}", offset, size, toString(file_segments))); return file_segments; } @@ -376,6 +381,7 @@ void FileCache::fillHolesWithEmptyFileSegments( LockedKey & locked_key, FileSegments & file_segments, const FileSegment::Range & range, + size_t non_aligned_right_offset, size_t file_segments_limit, bool fill_with_detached_file_segments, const CreateFileSegmentSettings & create_settings) @@ -442,7 +448,7 @@ void FileCache::fillHolesWithEmptyFileSegments( } else { - auto ranges = splitRange(current_pos, hole_size); + auto ranges = splitRange(current_pos, hole_size, hole_size); FileSegments hole; for (const auto & r : ranges) { @@ -479,7 +485,7 @@ void FileCache::fillHolesWithEmptyFileSegments( chassert(!file_segments_limit || file_segments.size() < file_segments_limit); - if (current_pos <= range.right) + if (current_pos <= non_aligned_right_offset) { /// ________] -- requested range /// _____] @@ -487,6 +493,7 @@ void FileCache::fillHolesWithEmptyFileSegments( /// segmentN auto hole_size = range.right - current_pos + 1; + auto non_aligned_size = non_aligned_right_offset - current_pos + 1; if (fill_with_detached_file_segments) { @@ -497,7 +504,7 @@ void FileCache::fillHolesWithEmptyFileSegments( } else { - auto ranges = splitRange(current_pos, hole_size); + auto ranges = splitRange(current_pos, non_aligned_size, hole_size); FileSegments hole; for (const auto & r : ranges) { @@ -542,7 +549,7 @@ FileSegmentsHolderPtr FileCache::set( else { file_segments = splitRangeIntoFileSegments( - *locked_key, offset, size, FileSegment::State::EMPTY, /* file_segments_limit */0, create_settings); + *locked_key, offset, size, size, FileSegment::State::EMPTY, /* file_segments_limit */0, create_settings); } return std::make_unique(std::move(file_segments)); @@ -659,9 +666,13 @@ FileCache::getOrSet( } } + chassert(range.left >= aligned_offset); + if (file_segments.empty()) { - file_segments = splitRangeIntoFileSegments(*locked_key, range.left, range.size(), FileSegment::State::EMPTY, file_segments_limit, create_settings); + file_segments = splitRangeIntoFileSegments( + *locked_key, range.left, /* size */offset + size - range.left, /* aligned_size */range.size(), + FileSegment::State::EMPTY, file_segments_limit, create_settings); } else { @@ -669,9 +680,9 @@ FileCache::getOrSet( chassert(file_segments.back()->range().left <= range.right); fillHolesWithEmptyFileSegments( - *locked_key, file_segments, range, file_segments_limit, /* fill_with_detached */false, create_settings); + *locked_key, file_segments, range, offset + size - 1, file_segments_limit, /* fill_with_detached */false, create_settings); - if (!file_segments.front()->range().contains(offset)) + if (!file_segments.front()->range().contains(range.left)) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected {} to include {} " "(end offset: {}, aligned offset: {}, aligned end offset: {})", @@ -713,7 +724,7 @@ FileSegmentsHolderPtr FileCache::get( } fillHolesWithEmptyFileSegments( - *locked_key, file_segments, range, file_segments_limit, /* fill_with_detached */true, CreateFileSegmentSettings{}); + *locked_key, file_segments, range, offset + size - 1, file_segments_limit, /* fill_with_detached */true, CreateFileSegmentSettings{}); chassert(!file_segments_limit || file_segments.size() <= file_segments_limit); return std::make_unique(std::move(file_segments)); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 527fd9d5edf..3f7eec73b56 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -263,7 +263,7 @@ private: /// Split range into subranges by max_file_segment_size, /// each subrange size must be less or equal to max_file_segment_size. - std::vector splitRange(size_t offset, size_t size); + std::vector splitRange(size_t offset, size_t size, size_t aligned_size); /// Split range into subranges by max_file_segment_size (same as in splitRange()) /// and create a new file segment for each subrange. @@ -273,6 +273,7 @@ private: LockedKey & locked_key, size_t offset, size_t size, + size_t aligned_size, FileSegment::State state, size_t file_segments_limit, const CreateFileSegmentSettings & create_settings); @@ -281,6 +282,7 @@ private: LockedKey & locked_key, FileSegments & file_segments, const FileSegment::Range & range, + size_t non_aligned_right_offset, size_t file_segments_limit, bool fill_with_detached_file_segments, const CreateFileSegmentSettings & settings); diff --git a/src/Interpreters/Cache/FileSegment.cpp b/src/Interpreters/Cache/FileSegment.cpp index 1664a91b694..c46fb978ae4 100644 --- a/src/Interpreters/Cache/FileSegment.cpp +++ b/src/Interpreters/Cache/FileSegment.cpp @@ -1008,7 +1008,12 @@ FileSegment & FileSegmentsHolder::add(FileSegmentPtr && file_segment) return *file_segments.back(); } -String FileSegmentsHolder::toString() +String FileSegmentsHolder::toString(bool with_state) +{ + return DB::toString(file_segments, with_state); +} + +String toString(const FileSegments & file_segments, bool with_state) { String ranges; for (const auto & file_segment : file_segments) @@ -1018,6 +1023,8 @@ String FileSegmentsHolder::toString() ranges += file_segment->range().toString(); if (file_segment->isUnbound()) ranges += "(unbound)"; + if (with_state) + ranges += "(" + FileSegment::stateToString(file_segment->state()) + ")"; } return ranges; } diff --git a/src/Interpreters/Cache/FileSegment.h b/src/Interpreters/Cache/FileSegment.h index d6b37b60dc1..25ffb880b45 100644 --- a/src/Interpreters/Cache/FileSegment.h +++ b/src/Interpreters/Cache/FileSegment.h @@ -291,7 +291,7 @@ struct FileSegmentsHolder : private boost::noncopyable size_t size() const { return file_segments.size(); } - String toString(); + String toString(bool with_state = false); void popFront() { completeAndPopFrontImpl(); } @@ -317,4 +317,6 @@ private: using FileSegmentsHolderPtr = std::unique_ptr; +String toString(const FileSegments & file_segments, bool with_state = false); + } diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 7a9b579c00a..4daa64b520d 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -19,7 +19,8 @@ cache s3_disk s3_cache/ - 104857600 + 100Mi + 5Mi 1 100 LRU From db0bce33526abf16e705b9e56d178d6e2c45a36b Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Aug 2024 15:01:15 +0200 Subject: [PATCH 13/27] Try make the code more understandable --- src/Interpreters/Cache/FileCache.cpp | 164 ++++++++++++------------- src/Interpreters/Cache/FileCache.h | 12 +- tests/config/config.d/storage_conf.xml | 2 +- 3 files changed, 80 insertions(+), 98 deletions(-) diff --git a/src/Interpreters/Cache/FileCache.cpp b/src/Interpreters/Cache/FileCache.cpp index 0a03f5dcc7d..4c17afb79be 100644 --- a/src/Interpreters/Cache/FileCache.cpp +++ b/src/Interpreters/Cache/FileCache.cpp @@ -318,7 +318,29 @@ FileSegments FileCache::getImpl(const LockedKey & locked_key, const FileSegment: std::vector FileCache::splitRange(size_t offset, size_t size, size_t aligned_size) { - assert(size > 0); + chassert(size > 0); + chassert(size <= aligned_size); + + /// Consider this example to understand why we need to account here for both `size` and `aligned_size`. + /// [________________]__________________] <-- requested range + /// ^ ^ + /// right offset aligned_right_offset + /// [_________] <-- last cached file segment, e.g. we have uncovered suffix of the requested range + /// [________________] + /// size + /// [____________________________________] + /// aligned_size + /// + /// So it is possible that we split this hole range into sub-segments by `max_file_segment_size` + /// and get something like this: + /// + /// [________________________] + /// ^ ^ + /// right_offset right_offset + max_file_segment_size + /// e.g. there is no need to create sub-segment for range (right_offset + max_file_segment_size, aligned_right_offset]. + /// Because its left offset would be bigger than right_offset. + /// Therefore, we set end_pos_non_included as offset+size, but remaining_size as aligned_size. + std::vector ranges; size_t current_pos = offset; @@ -339,42 +361,23 @@ std::vector FileCache::splitRange(size_t offset, size_t size return ranges; } -FileSegments FileCache::splitRangeIntoFileSegments( +FileSegments FileCache::createFileSegmentsFromRanges( LockedKey & locked_key, - size_t offset, - size_t size, - size_t aligned_size, - FileSegment::State state, + const std::vector & ranges, + size_t & file_segments_count, size_t file_segments_limit, const CreateFileSegmentSettings & create_settings) { - chassert(size > 0); - chassert(size <= aligned_size); - /// We take `size` as a soft limit and `aligned_size` as a hard limit. - - auto current_pos = offset; - auto end_pos_non_included = offset + size; - - size_t current_file_segment_size; - size_t remaining_size = aligned_size; - - FileSegments file_segments; - const size_t max_size = max_file_segment_size.load(); - while (current_pos < end_pos_non_included && (!file_segments_limit || file_segments.size() < file_segments_limit)) + FileSegments result; + for (const auto & r : ranges) { - current_file_segment_size = std::min(remaining_size, max_size); - remaining_size -= current_file_segment_size; - - auto file_segment_metadata_it = addFileSegment( - locked_key, current_pos, current_file_segment_size, state, create_settings, nullptr); - file_segments.push_back(file_segment_metadata_it->second->file_segment); - - current_pos += current_file_segment_size; + if (file_segments_limit && file_segments_count >= file_segments_limit) + break; + auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, create_settings, nullptr); + result.push_back(metadata_it->second->file_segment); + ++file_segments_count; } - - chassert(file_segments.size() == file_segments_limit || file_segments.back()->range().contains(offset + size - 1), - fmt::format("Offset: {}, size: {}, file segments: {}", offset, size, toString(file_segments))); - return file_segments; + return result; } void FileCache::fillHolesWithEmptyFileSegments( @@ -448,18 +451,9 @@ void FileCache::fillHolesWithEmptyFileSegments( } else { - auto ranges = splitRange(current_pos, hole_size, hole_size); - FileSegments hole; - for (const auto & r : ranges) - { - auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, create_settings, nullptr); - hole.push_back(metadata_it->second->file_segment); - ++processed_count; - - if (is_limit_reached()) - break; - } - file_segments.splice(it, std::move(hole)); + const auto ranges = splitRange(current_pos, hole_size, hole_size); + auto hole_segments = createFileSegmentsFromRanges(locked_key, ranges, processed_count, file_segments_limit, create_settings); + file_segments.splice(it, std::move(hole_segments)); } if (is_limit_reached()) @@ -493,29 +487,20 @@ void FileCache::fillHolesWithEmptyFileSegments( /// segmentN auto hole_size = range.right - current_pos + 1; - auto non_aligned_size = non_aligned_right_offset - current_pos + 1; + auto non_aligned_hole_size = non_aligned_right_offset - current_pos + 1; if (fill_with_detached_file_segments) { auto file_segment = std::make_shared( - locked_key.getKey(), current_pos, hole_size, FileSegment::State::DETACHED, create_settings); + locked_key.getKey(), current_pos, non_aligned_hole_size, FileSegment::State::DETACHED, create_settings); file_segments.insert(file_segments.end(), file_segment); } else { - auto ranges = splitRange(current_pos, non_aligned_size, hole_size); - FileSegments hole; - for (const auto & r : ranges) - { - auto metadata_it = addFileSegment(locked_key, r.left, r.size(), FileSegment::State::EMPTY, create_settings, nullptr); - hole.push_back(metadata_it->second->file_segment); - ++processed_count; - - if (is_limit_reached()) - break; - } - file_segments.splice(it, std::move(hole)); + const auto ranges = splitRange(current_pos, non_aligned_hole_size, hole_size); + auto hole_segments = createFileSegmentsFromRanges(locked_key, ranges, processed_count, file_segments_limit, create_settings); + file_segments.splice(it, std::move(hole_segments)); if (is_limit_reached()) erase_unprocessed(); @@ -548,8 +533,9 @@ FileSegmentsHolderPtr FileCache::set( } else { - file_segments = splitRangeIntoFileSegments( - *locked_key, offset, size, size, FileSegment::State::EMPTY, /* file_segments_limit */0, create_settings); + const auto ranges = splitRange(offset, size, size); + size_t file_segments_count = 0; + file_segments = createFileSegmentsFromRanges(*locked_key, ranges, file_segments_count, /* file_segments_limit */0, create_settings); } return std::make_unique(std::move(file_segments)); @@ -569,23 +555,27 @@ FileCache::getOrSet( assertInitialized(); - FileSegment::Range range(offset, offset + size - 1); + FileSegment::Range initial_range(offset, offset + size - 1); + /// result_range is initial range, which will be adjusted according to + /// 1. aligned offset, alighed_end_offset + /// 2. max_file_segments_limit + FileSegment::Range result_range = initial_range; - const auto aligned_offset = roundDownToMultiple(range.left, boundary_alignment); - auto aligned_end_offset = std::min(roundUpToMultiple(offset + size, boundary_alignment), file_size) - 1; + const auto aligned_offset = roundDownToMultiple(initial_range.left, boundary_alignment); + auto aligned_end_offset = std::min(roundUpToMultiple(initial_range.right + 1, boundary_alignment), file_size) - 1; - chassert(aligned_offset <= range.left); - chassert(aligned_end_offset >= range.right); + chassert(aligned_offset <= initial_range.left); + chassert(aligned_end_offset >= initial_range.right); auto locked_key = metadata.lockKeyMetadata(key, CacheMetadata::KeyNotFoundPolicy::CREATE_EMPTY, user); /// Get all segments which intersect with the given range. - auto file_segments = getImpl(*locked_key, range, file_segments_limit); + auto file_segments = getImpl(*locked_key, initial_range, file_segments_limit); if (file_segments_limit) { chassert(file_segments.size() <= file_segments_limit); if (file_segments.size() == file_segments_limit) - range.right = aligned_end_offset = file_segments.back()->range().right; + result_range.right = aligned_end_offset = file_segments.back()->range().right; } /// Check case if we have uncovered prefix, e.g. @@ -597,11 +587,11 @@ FileCache::getOrSet( /// [ ] /// ^----^ /// uncovered prefix. - const bool has_uncovered_prefix = file_segments.empty() || range.left < file_segments.front()->range().left; + const bool has_uncovered_prefix = file_segments.empty() || result_range.left < file_segments.front()->range().left; - if (aligned_offset < range.left && has_uncovered_prefix) + if (aligned_offset < result_range.left && has_uncovered_prefix) { - auto prefix_range = FileSegment::Range(aligned_offset, file_segments.empty() ? range.left - 1 : file_segments.front()->range().left - 1); + auto prefix_range = FileSegment::Range(aligned_offset, file_segments.empty() ? result_range.left - 1 : file_segments.front()->range().left - 1); auto prefix_file_segments = getImpl(*locked_key, prefix_range, /* file_segments_limit */0); if (prefix_file_segments.empty()) @@ -610,7 +600,7 @@ FileCache::getOrSet( /// ^ ^ ^ /// aligned_offset range.left range.right /// [___] [__________] <-- current cache (example) - range.left = aligned_offset; + result_range.left = aligned_offset; } else { @@ -621,10 +611,10 @@ FileCache::getOrSet( /// ^ /// prefix_file_segments.back().right - chassert(prefix_file_segments.back()->range().right < range.left); + chassert(prefix_file_segments.back()->range().right < result_range.left); chassert(prefix_file_segments.back()->range().right >= aligned_offset); - range.left = prefix_file_segments.back()->range().right + 1; + result_range.left = prefix_file_segments.back()->range().right + 1; } } @@ -637,11 +627,11 @@ FileCache::getOrSet( /// [___] /// ^---^ /// uncovered_suffix - const bool has_uncovered_suffix = file_segments.empty() || file_segments.back()->range().right < range.right; + const bool has_uncovered_suffix = file_segments.empty() || file_segments.back()->range().right < result_range.right; - if (range.right < aligned_end_offset && has_uncovered_suffix) + if (result_range.right < aligned_end_offset && has_uncovered_suffix) { - auto suffix_range = FileSegment::Range(range.right, aligned_end_offset); + auto suffix_range = FileSegment::Range(result_range.right, aligned_end_offset); /// We need to get 1 file segment, so file_segments_limit = 1 here. auto suffix_file_segments = getImpl(*locked_key, suffix_range, /* file_segments_limit */1); @@ -652,7 +642,7 @@ FileCache::getOrSet( /// range.left range.right aligned_end_offset /// [___] [___] <-- current cache (example) - range.right = aligned_end_offset; + result_range.right = aligned_end_offset; } else { @@ -662,35 +652,33 @@ FileCache::getOrSet( /// [___] [___] [_________] <-- current cache (example) /// ^ /// suffix_file_segments.front().left - range.right = suffix_file_segments.front()->range().left - 1; + result_range.right = suffix_file_segments.front()->range().left - 1; } } - chassert(range.left >= aligned_offset); - if (file_segments.empty()) { - file_segments = splitRangeIntoFileSegments( - *locked_key, range.left, /* size */offset + size - range.left, /* aligned_size */range.size(), - FileSegment::State::EMPTY, file_segments_limit, create_settings); + auto ranges = splitRange(result_range.left, initial_range.size() + (initial_range.left - result_range.left), result_range.size()); + size_t file_segments_count = file_segments.size(); + file_segments.splice(file_segments.end(), createFileSegmentsFromRanges(*locked_key, ranges, file_segments_count, file_segments_limit, create_settings)); } else { - chassert(file_segments.front()->range().right >= range.left); - chassert(file_segments.back()->range().left <= range.right); + chassert(file_segments.front()->range().right >= result_range.left); + chassert(file_segments.back()->range().left <= result_range.right); fillHolesWithEmptyFileSegments( - *locked_key, file_segments, range, offset + size - 1, file_segments_limit, /* fill_with_detached */false, create_settings); + *locked_key, file_segments, result_range, offset + size - 1, file_segments_limit, /* fill_with_detached */false, create_settings); - if (!file_segments.front()->range().contains(range.left)) + if (!file_segments.front()->range().contains(result_range.left)) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected {} to include {} " "(end offset: {}, aligned offset: {}, aligned end offset: {})", - file_segments.front()->range().toString(), offset, range.right, aligned_offset, aligned_end_offset); + file_segments.front()->range().toString(), offset, result_range.right, aligned_offset, aligned_end_offset); } } - chassert(file_segments_limit ? file_segments.back()->range().left <= range.right : file_segments.back()->range().contains(range.right)); + chassert(file_segments_limit ? file_segments.back()->range().left <= result_range.right : file_segments.back()->range().contains(result_range.right)); chassert(!file_segments_limit || file_segments.size() <= file_segments_limit); return std::make_unique(std::move(file_segments)); diff --git a/src/Interpreters/Cache/FileCache.h b/src/Interpreters/Cache/FileCache.h index 3f7eec73b56..07be802a940 100644 --- a/src/Interpreters/Cache/FileCache.h +++ b/src/Interpreters/Cache/FileCache.h @@ -265,16 +265,10 @@ private: /// each subrange size must be less or equal to max_file_segment_size. std::vector splitRange(size_t offset, size_t size, size_t aligned_size); - /// Split range into subranges by max_file_segment_size (same as in splitRange()) - /// and create a new file segment for each subrange. - /// If `file_segments_limit` > 0, create no more than first file_segments_limit - /// file segments. - FileSegments splitRangeIntoFileSegments( + FileSegments createFileSegmentsFromRanges( LockedKey & locked_key, - size_t offset, - size_t size, - size_t aligned_size, - FileSegment::State state, + const std::vector & ranges, + size_t & file_segments_count, size_t file_segments_limit, const CreateFileSegmentSettings & create_settings); diff --git a/tests/config/config.d/storage_conf.xml b/tests/config/config.d/storage_conf.xml index 4daa64b520d..e106e3a0e6b 100644 --- a/tests/config/config.d/storage_conf.xml +++ b/tests/config/config.d/storage_conf.xml @@ -19,7 +19,7 @@ cache s3_disk s3_cache/ - 100Mi + 104857600 5Mi 1 100 From 9c7464e0653782af385dbc884dd3acecfc69c6cc Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 2 Aug 2024 16:04:11 +0200 Subject: [PATCH 14/27] Stateless tests: reduce pure_http_client timeout to get reasons of timed out tests --- tests/queries/0_stateless/helpers/pure_http_client.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/helpers/pure_http_client.py b/tests/queries/0_stateless/helpers/pure_http_client.py index 0e7a4d27f4f..7a8efec36bb 100644 --- a/tests/queries/0_stateless/helpers/pure_http_client.py +++ b/tests/queries/0_stateless/helpers/pure_http_client.py @@ -18,7 +18,7 @@ class ClickHouseClient: self.host = host def query( - self, query, connection_timeout=1500, settings=dict(), binary_result=False + self, query, connection_timeout=500, settings=dict(), binary_result=False ): NUMBER_OF_TRIES = 30 DELAY = 10 @@ -47,12 +47,12 @@ class ClickHouseClient: else: raise ValueError(r.text) - def query_return_df(self, query, connection_timeout=1500): + def query_return_df(self, query, connection_timeout=500): data = self.query(query, connection_timeout) df = pd.read_csv(io.StringIO(data), sep="\t") return df - def query_with_data(self, query, data, connection_timeout=1500, settings=dict()): + def query_with_data(self, query, data, connection_timeout=500, settings=dict()): params = { "query": query, "timeout_before_checking_execution_speed": 120, From 2c9cef38e56c65ec9bbe7f3af21d4865662f6e9a Mon Sep 17 00:00:00 2001 From: Nikita Fomichev Date: Fri, 2 Aug 2024 16:05:39 +0200 Subject: [PATCH 15/27] Stateless tests: fix hanging tests `02473_multistep_prewhere*` `00411_long_accurate_number_comparison*` --- ...411_long_accurate_number_comparison.python | 19 ++++++------------- .../02473_multistep_prewhere.python | 4 ++-- .../02473_multistep_split_prewhere.python | 4 ++-- .../0_stateless/helpers/pure_http_client.py | 17 ++++++++++++++++- 4 files changed, 26 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/00411_long_accurate_number_comparison.python b/tests/queries/0_stateless/00411_long_accurate_number_comparison.python index 045de9ee7ee..38b108a696f 100644 --- a/tests/queries/0_stateless/00411_long_accurate_number_comparison.python +++ b/tests/queries/0_stateless/00411_long_accurate_number_comparison.python @@ -2,23 +2,16 @@ import os, itertools, urllib.request, urllib.parse, urllib.error, urllib.request, urllib.error, urllib.parse, sys +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) -def get_ch_answer(query): - return ( - urllib.request.urlopen( - os.environ.get( - "CLICKHOUSE_URL", - "http://localhost:" + os.environ.get("CLICKHOUSE_PORT_HTTP", "8123"), - ), - data=query.encode(), - ) - .read() - .decode() - ) +from pure_http_client import ClickHouseClient + +client = ClickHouseClient() def check_answers(query, answer): - ch_answer = get_ch_answer(query) + ch_answer = client.query(query) if ch_answer.strip() != answer.strip(): print("FAIL on query:", query) print("Expected answer:", answer) diff --git a/tests/queries/0_stateless/02473_multistep_prewhere.python b/tests/queries/0_stateless/02473_multistep_prewhere.python index 11095202039..09326b6365d 100644 --- a/tests/queries/0_stateless/02473_multistep_prewhere.python +++ b/tests/queries/0_stateless/02473_multistep_prewhere.python @@ -6,7 +6,7 @@ import sys CURDIR = os.path.dirname(os.path.realpath(__file__)) sys.path.insert(0, os.path.join(CURDIR, "helpers")) -from pure_http_client import ClickHouseClient +from pure_http_client import ClickHouseClient, requests_session_with_retries class Tester: @@ -195,7 +195,7 @@ def main(): default_index_granularity = 10 total_rows = 7 * default_index_granularity step = default_index_granularity - session = requests.Session() + session = requests_session_with_retries() for index_granularity in [ default_index_granularity - 1, default_index_granularity, diff --git a/tests/queries/0_stateless/02473_multistep_split_prewhere.python b/tests/queries/0_stateless/02473_multistep_split_prewhere.python index 19444994fd2..10e94059171 100644 --- a/tests/queries/0_stateless/02473_multistep_split_prewhere.python +++ b/tests/queries/0_stateless/02473_multistep_split_prewhere.python @@ -6,7 +6,7 @@ import sys CURDIR = os.path.dirname(os.path.realpath(__file__)) sys.path.insert(0, os.path.join(CURDIR, "helpers")) -from pure_http_client import ClickHouseClient +from pure_http_client import ClickHouseClient, requests_session_with_retries class Tester: @@ -161,7 +161,7 @@ def main(): default_index_granularity = 10 total_rows = 8 * default_index_granularity step = default_index_granularity - session = requests.Session() + session = requests_session_with_retries() for index_granularity in [default_index_granularity - 1, default_index_granularity]: tester = Tester(session, url, index_granularity, total_rows) # Test combinations of ranges of columns c and d diff --git a/tests/queries/0_stateless/helpers/pure_http_client.py b/tests/queries/0_stateless/helpers/pure_http_client.py index 7a8efec36bb..a31a91e0550 100644 --- a/tests/queries/0_stateless/helpers/pure_http_client.py +++ b/tests/queries/0_stateless/helpers/pure_http_client.py @@ -1,7 +1,8 @@ import os import io -import sys import requests +from requests.adapters import HTTPAdapter +from requests.packages.urllib3.util.retry import Retry import time import pandas as pd @@ -77,3 +78,17 @@ class ClickHouseClient: return result else: raise ValueError(r.text) + + +def requests_session_with_retries(retries=3, timeout=180): + session = requests.Session() + retry = Retry( + total=retries, + read=retries, + connect=retries, + ) + adapter = HTTPAdapter(max_retries=retry) + session.mount("http://", adapter) + session.mount("https://", adapter) + session.timeout = timeout + return session From f06ae2f5518ff8cb610b337d4900fd6f0088190f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 23:27:19 +0200 Subject: [PATCH 16/27] Fill only selected columns from system.clusters Some of them pretty heavy, i.e. is_active for ReplicatedDatabase This should fix 02903_rmt_retriable_merge_exception flakiness [1]. [1]: https://s3.amazonaws.com/clickhouse-test-reports/67687/89c47df559ba23d988f8af3c342e0c8d5531f4b8/fast_test.html Signed-off-by: Azat Khuzhin --- src/Storages/System/StorageSystemClusters.cpp | 82 ++++++++++++------- src/Storages/System/StorageSystemClusters.h | 6 +- 2 files changed, 56 insertions(+), 32 deletions(-) diff --git a/src/Storages/System/StorageSystemClusters.cpp b/src/Storages/System/StorageSystemClusters.cpp index 160c8d6270e..9c5c07ae49f 100644 --- a/src/Storages/System/StorageSystemClusters.cpp +++ b/src/Storages/System/StorageSystemClusters.cpp @@ -40,10 +40,10 @@ ColumnsDescription StorageSystemClusters::getColumnsDescription() return description; } -void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const +void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector columns_mask) const { for (const auto & name_and_cluster : context->getClusters()) - writeCluster(res_columns, name_and_cluster, {}); + writeCluster(res_columns, columns_mask, name_and_cluster, /* replicated= */ nullptr); const auto databases = DatabaseCatalog::instance().getDatabases(); for (const auto & name_and_database : databases) @@ -52,18 +52,15 @@ void StorageSystemClusters::fillData(MutableColumns & res_columns, ContextPtr co { if (auto database_cluster = replicated->tryGetCluster()) - writeCluster(res_columns, {name_and_database.first, database_cluster}, - replicated->tryGetAreReplicasActive(database_cluster)); + writeCluster(res_columns, columns_mask, {name_and_database.first, database_cluster}, replicated); if (auto database_cluster = replicated->tryGetAllGroupsCluster()) - writeCluster(res_columns, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + name_and_database.first, database_cluster}, - replicated->tryGetAreReplicasActive(database_cluster)); + writeCluster(res_columns, columns_mask, {DatabaseReplicated::ALL_GROUPS_CLUSTER_PREFIX + name_and_database.first, database_cluster}, replicated); } } } -void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, - const std::vector & is_active) +void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const std::vector & columns_mask, const NameAndCluster & name_and_cluster, const DatabaseReplicated * replicated) { const String & cluster_name = name_and_cluster.first; const ClusterPtr & cluster = name_and_cluster.second; @@ -79,30 +76,55 @@ void StorageSystemClusters::writeCluster(MutableColumns & res_columns, const Nam for (size_t replica_index = 0; replica_index < shard_addresses.size(); ++replica_index) { - size_t i = 0; + size_t src_index = 0, res_index = 0; const auto & address = shard_addresses[replica_index]; - res_columns[i++]->insert(cluster_name); - res_columns[i++]->insert(shard_info.shard_num); - res_columns[i++]->insert(shard_info.weight); - res_columns[i++]->insert(shard_info.has_internal_replication); - res_columns[i++]->insert(replica_index + 1); - res_columns[i++]->insert(address.host_name); - auto resolved = address.getResolvedAddress(); - res_columns[i++]->insert(resolved ? resolved->host().toString() : String()); - res_columns[i++]->insert(address.port); - res_columns[i++]->insert(address.is_local); - res_columns[i++]->insert(address.user); - res_columns[i++]->insert(address.default_database); - res_columns[i++]->insert(pool_status[replica_index].error_count); - res_columns[i++]->insert(pool_status[replica_index].slowdown_count); - res_columns[i++]->insert(pool_status[replica_index].estimated_recovery_time.count()); - res_columns[i++]->insert(address.database_shard_name); - res_columns[i++]->insert(address.database_replica_name); - if (is_active.empty()) - res_columns[i++]->insertDefault(); - else - res_columns[i++]->insert(is_active[replica_idx++]); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(cluster_name); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(shard_info.shard_num); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(shard_info.weight); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(shard_info.has_internal_replication); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(replica_index + 1); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.host_name); + if (columns_mask[src_index++]) + { + auto resolved = address.getResolvedAddress(); + res_columns[res_index++]->insert(resolved ? resolved->host().toString() : String()); + } + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.port); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.is_local); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.user); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.default_database); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(pool_status[replica_index].error_count); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(pool_status[replica_index].slowdown_count); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(pool_status[replica_index].estimated_recovery_time.count()); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.database_shard_name); + if (columns_mask[src_index++]) + res_columns[res_index++]->insert(address.database_replica_name); + if (columns_mask[src_index++]) + { + std::vector is_active; + if (replicated) + is_active = replicated->tryGetAreReplicasActive(name_and_cluster.second); + + if (is_active.empty()) + res_columns[res_index++]->insertDefault(); + else + res_columns[res_index++]->insert(is_active[replica_idx++]); + } } } } diff --git a/src/Storages/System/StorageSystemClusters.h b/src/Storages/System/StorageSystemClusters.h index 0f7c792261d..f6adb902f43 100644 --- a/src/Storages/System/StorageSystemClusters.h +++ b/src/Storages/System/StorageSystemClusters.h @@ -10,6 +10,7 @@ namespace DB class Context; class Cluster; +class DatabaseReplicated; /** Implements system table 'clusters' * that allows to obtain information about available clusters @@ -26,8 +27,9 @@ protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; using NameAndCluster = std::pair>; - void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector) const override; - static void writeCluster(MutableColumns & res_columns, const NameAndCluster & name_and_cluster, const std::vector & is_active); + void fillData(MutableColumns & res_columns, ContextPtr context, const ActionsDAG::Node *, std::vector columns_mask) const override; + static void writeCluster(MutableColumns & res_columns, const std::vector & columns_mask, const NameAndCluster & name_and_cluster, const DatabaseReplicated * replicated); + bool supportsColumnsMask() const override { return true; } }; } From 9d0e066cda8d0ccb6bd4f9e07fee36a2bfae707a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 3 Aug 2024 23:41:10 +0200 Subject: [PATCH 17/27] Bump NuRaft (to properly catch thread exceptions) Refs: https://github.com/ClickHouse/NuRaft/pull/75 Refs: https://github.com/eBay/NuRaft/pull/525 Signed-off-by: Azat Khuzhin --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index cb5dc3c906e..c2b0811f164 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit cb5dc3c906e80f253e9ce9535807caef827cc2e0 +Subproject commit c2b0811f164a7948208489562dab4f186eb305ce From 59e5a09b491d78eff188ecd685bb6ca3769831ce Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 09:02:19 +0200 Subject: [PATCH 18/27] Use RabbitMQ without management in tests (attempt to improve startup) Sometimes startup can take ~90 seconds [1]: 2024-08-03 23:11:38.756067+00:00 [info] <0.9.0> Time to start RabbitMQ: 94651980 us Unlike normally ~10 [2]: 2024-08-04 00:33:07.016137+00:00 [info] <0.9.0> Time to start RabbitMQ: 10082489 us [1]: https://s3.amazonaws.com/clickhouse-test-reports/67737/b4e3bbcb82158bea4f5db1d9f5c28cfb741d1d51/integration_tests__asan__old_analyzer__[4_6].html [2]: https://s3.amazonaws.com/clickhouse-test-reports/66671/2f00c962711e13ca00af324366421fe4593b4ce6/integration_tests__asan__old_analyzer__[4_6].html I've tried locally, the difference is very small, 3135665 us (+management) vs (2740747 us), but still something, and who knows how it works under pressure. Signed-off-by: Azat Khuzhin --- tests/integration/compose/docker_compose_rabbitmq.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/compose/docker_compose_rabbitmq.yml b/tests/integration/compose/docker_compose_rabbitmq.yml index 61b21e0e3d9..94c7f0111c4 100644 --- a/tests/integration/compose/docker_compose_rabbitmq.yml +++ b/tests/integration/compose/docker_compose_rabbitmq.yml @@ -2,7 +2,7 @@ version: '2.3' services: rabbitmq1: - image: rabbitmq:3.12.6-management-alpine + image: rabbitmq:3.12.6-alpine hostname: rabbitmq1 expose: - ${RABBITMQ_PORT:-5672} From 264be9c598b42d91ee0a19f718c4d9a4291c7bc4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 09:06:50 +0200 Subject: [PATCH 19/27] tests: increase timeout for RabbitMQ startup Signed-off-by: Azat Khuzhin --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 9259c720ff0..a1bdee33d57 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2371,7 +2371,7 @@ class ClickHouseCluster: time.sleep(0.5) raise Exception("Cannot wait PostgreSQL Java Client container") - def wait_rabbitmq_to_start(self, timeout=30): + def wait_rabbitmq_to_start(self, timeout=60): self.print_all_docker_pieces() self.rabbitmq_ip = self.get_instance_ip(self.rabbitmq_host) From dc527b6fd1dfdffb1d177237bffc69cd110cd2a7 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 09:07:05 +0200 Subject: [PATCH 20/27] tests: detailed errors for RabbitMQ startup Signed-off-by: Azat Khuzhin --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index a1bdee33d57..56d111629c9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2399,7 +2399,7 @@ class ClickHouseCluster: ) rabbitmq_debuginfo(self.rabbitmq_docker_id, self.rabbitmq_cookie) except Exception as e: - logging.debug("Unable to get logs from docker.") + logging.debug(f"Unable to get logs from docker: {e}.") raise Exception("Cannot wait RabbitMQ container") def wait_nats_is_available(self, max_retries=5): From f0aaac3bd19c21a796f54ba080fd67f92959131a Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 14:14:28 +0200 Subject: [PATCH 21/27] tests: remove useless retries from test_ttl_move::test_alter_with_merge_work Signed-off-by: Azat Khuzhin --- tests/integration/test_ttl_move/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 3b79ea7916d..48a6224347d 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -76,7 +76,7 @@ def get_used_disks_for_table(node, table_name, partition=None): ) -def check_used_disks_with_retry(node, table_name, expected_disks, retries): +def check_used_disks_with_retry(node, table_name, expected_disks, retries=1): for _ in range(retries): used_disks = get_used_disks_for_table(node, table_name) if set(used_disks).issubset(expected_disks): @@ -1635,9 +1635,9 @@ def test_alter_with_merge_work(started_cluster, name, engine, positive): optimize_table(20) if positive: - assert check_used_disks_with_retry(node1, name, set(["external"]), 100) + assert check_used_disks_with_retry(node1, name, set(["external"])) else: - assert check_used_disks_with_retry(node1, name, set(["jbod1", "jbod2"]), 50) + assert check_used_disks_with_retry(node1, name, set(["jbod1", "jbod2"])) time.sleep(5) From 47dbc5e05b12213a08c25ade9536603a3fd2b175 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 14:16:17 +0200 Subject: [PATCH 22/27] tests: add debug info into test_ttl_move::test_alter_with_merge_work Signed-off-by: Azat Khuzhin --- tests/integration/test_ttl_move/test.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 48a6224347d..4ebe9a30699 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1635,9 +1635,17 @@ def test_alter_with_merge_work(started_cluster, name, engine, positive): optimize_table(20) if positive: - assert check_used_disks_with_retry(node1, name, set(["external"])) + assert check_used_disks_with_retry( + node1, name, set(["external"]) + ), "Parts: " + node1.query( + f"SELECT disk_name, name FROM system.parts WHERE table = '{name}' AND active = 1" + ) else: - assert check_used_disks_with_retry(node1, name, set(["jbod1", "jbod2"])) + assert check_used_disks_with_retry( + node1, name, set(["jbod1", "jbod2"]) + ), "Parts: " + node1.query( + f"SELECT disk_name, name FROM system.parts WHERE table = '{name}' AND active = 1" + ) time.sleep(5) From 062490e1b40a8df8d63fca567b11e7dd26cf52ee Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 14:22:24 +0200 Subject: [PATCH 23/27] tests: fix test_ttl_move::test_alter_with_merge_work flakiness MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Increase timeout for TTL DELETE, since otherwise if other routines will take too long, the part will be removed when it should be still be on "external" disk: 2024.08.04 03:48:53.803032 [ 622 ] {} default.mt_test_alter_with_merge_work_1722743323 (9dc6904a-f082-4f06-be7a-efe4733e811c): Will drop empty part all_1_3_4_4 And this is how part_log looks like: SELECT event_time, event_type, rows, part_name, error, database, disk_name FROM system.part_log WHERE `table` = 'mt_test_alter_with_merge_work_1722743323' ORDER BY event_time ASC Query id: a118b3cd-e4fe-45a5-b675-d73bdd887d79 ┌──────────event_time─┬─event_type─┬─rows─┬─part_name───┬─error─┬─database─┬─disk_name─┐ 1. │ 2024-08-04 03:48:44 │ NewPart │ 2 │ all_1_1_0 │ 0 │ default │ jbod1 │ 2. │ 2024-08-04 03:48:44 │ NewPart │ 2 │ all_2_2_0 │ 0 │ default │ jbod2 │ 3. │ 2024-08-04 03:48:45 │ NewPart │ 2 │ all_3_3_0 │ 0 │ default │ jbod1 │ 4. │ 2024-08-04 03:48:46 │ MutatePart │ 2 │ all_1_1_0_4 │ 0 │ default │ jbod1 │ 5. │ 2024-08-04 03:48:46 │ MutatePart │ 2 │ all_2_2_0_4 │ 0 │ default │ jbod2 │ 6. │ 2024-08-04 03:48:46 │ MutatePart │ 2 │ all_3_3_0_4 │ 0 │ default │ jbod1 │ 7. │ 2024-08-04 03:48:47 │ MovePart │ 2 │ all_1_1_0_4 │ 0 │ default │ external │ 8. │ 2024-08-04 03:48:47 │ MovePart │ 2 │ all_3_3_0_4 │ 0 │ default │ jbod2 │ 9. │ 2024-08-04 03:48:47 │ MergeParts │ 6 │ all_1_3_1_4 │ 0 │ default │ jbod2 │ 10. │ 2024-08-04 03:48:48 │ MovePart │ 6 │ all_1_3_1_4 │ 0 │ default │ external │ 11. │ 2024-08-04 03:48:52 │ MergeParts │ 4 │ all_1_3_2_4 │ 0 │ default │ external │ 12. │ 2024-08-04 03:48:53 │ MergeParts │ 0 │ all_1_3_3_4 │ 0 │ default │ external │ # rows==0 13. │ 2024-08-04 03:48:53 │ MergeParts │ 0 │ all_1_3_4_4 │ 0 │ default │ external │ └─────────────────────┴────────────┴──────┴─────────────┴───────┴──────────┴───────────┘ CI: https://s3.amazonaws.com/clickhouse-test-reports/66671/2f00c962711e13ca00af324366421fe4593b4ce6/integration_tests__tsan__[5_6].html Signed-off-by: Azat Khuzhin --- tests/integration/test_ttl_move/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 4ebe9a30699..925bdf9baaa 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -1613,7 +1613,7 @@ def test_alter_with_merge_work(started_cluster, name, engine, positive): ALTER TABLE {name} MODIFY TTL d1 + INTERVAL 0 SECOND TO DISK 'jbod2', d1 + INTERVAL 5 SECOND TO VOLUME 'external', - d1 + INTERVAL 10 SECOND DELETE + d1 + INTERVAL 30 SECOND DELETE """.format( name=name ) @@ -1647,7 +1647,7 @@ def test_alter_with_merge_work(started_cluster, name, engine, positive): f"SELECT disk_name, name FROM system.parts WHERE table = '{name}' AND active = 1" ) - time.sleep(5) + time.sleep(25) optimize_table(20) From 9f31488e502c2b2c02e3058f9794829aac14f8b9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 4 Aug 2024 15:43:18 +0200 Subject: [PATCH 24/27] Fix dictionary hang in case of CANNOT_SCHEDULE_TASK while loading On CI you can find that 01747_executable_pool_dictionary_implicit_key can hang [1], it is possible due to after CANNOT_SCHEDULE_TASK the async loading will hang: 2024.07.18 03:56:32.365226 [ 6138 ] {6206a18f-668c-4a5c-a5ad-07f577220762} ExternalDictionariesLoader: Will load the object 'executable_pool_simple_implicit_key' in background, force = false, loading_id = 2 2024.07.18 03:56:32.368005 [ 6138 ] {6206a18f-668c-4a5c-a5ad-07f577220762} executeQuery: Code: 439. DB::Exception: Cannot schedule a task: fault injected (threads=766, jobs=746): In scope SELECT dictGet('executable_pool_simple_implicit_key', 'a', toUInt64(1)). (CANNOT_SCHEDULE_TASK) (version 24.7.1.2241) (from [::1]:56446) (comment: 01747_executable_pool_dictionary_implicit_key.sql) (in query: SELECT dictGet('executable_pool_simple_implicit_key', 'a', toUInt64(1));), Stack trace (when copying this message, always include the lines below): 0. /build/contrib/llvm-project/libcxx/include/exception:141: Poco::Exception::Exception(String const&, int) @ 0x0000000015f8a292 1. /build/src/Common/Exception.cpp:110: DB::Exception::Exception(DB::Exception::MessageMasked&&, int, bool) @ 0x000000000c3df6b9 2. /build/contrib/llvm-project/libcxx/include/string:1499: DB::Exception::Exception(PreformattedMessage&&, int) @ 0x0000000006de714c 3. /build/contrib/llvm-project/libcxx/include/vector:438: DB::Exception::Exception(int, FormatStringHelperImpl::type, std::type_identity::type, std::type_identity::type>, String const&, unsigned long&&, unsigned long&) @ 0x000000000c4838eb 4. /build/src/Common/ThreadPool.cpp:0: void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool)::'lambda'(String const&)::operator()(String const&) const @ 0x000000000c4832d3 5. /build/src/Common/ThreadPool.cpp:186: void ThreadPoolImpl::scheduleImpl(std::function, Priority, std::optional, bool) @ 0x000000000c47e7db 6. /build/contrib/llvm-project/libcxx/include/__functional/function.h:818: ? @ 0x000000000c47ec8d 7. /build/contrib/llvm-project/libcxx/include/__functional/function.h:818: ? @ 0x000000001114b16e 8. /build/contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:701: DB::ExternalLoader::LoadingDispatcher::startLoading(DB::ExternalLoader::LoadingDispatcher::Info&, bool, unsigned long) @ 0x0000000011147733 9. /build/src/Interpreters/ExternalLoader.cpp:837: DB::ExternalLoader::LoadingDispatcher::loadImpl(String const&, std::chrono::duration>, bool, std::unique_lock&)::'lambda'()::operator()() const @ 0x0000000011158bf9 10. /build/contrib/llvm-project/libcxx/include/__mutex_base:397: DB::ExternalLoader::LoadingDispatcher::loadImpl(String const&, std::chrono::duration>, bool, std::unique_lock&) @ 0x00000000111588bc 11. /build/src/Interpreters/ExternalLoader.cpp:604: DB::ExternalLoader::LoadResult DB::ExternalLoader::LoadingDispatcher::tryLoad(String const&, std::chrono::duration>) @ 0x00000000111440bf 12. /build/src/Interpreters/ExternalLoader.cpp:1381: std::shared_ptr DB::ExternalLoader::load, void>(String const&) const @ 0x00000000111442f5 13. /build/contrib/llvm-project/libcxx/include/__memory/shared_ptr.h:587: DB::ExternalDictionariesLoader::getDictionary(String const&, std::shared_ptr) const @ 0x0000000011141028 14. /build/src/Functions/FunctionsExternalDictionaries.h:76: DB::FunctionDictHelper::getDictionary(String const&) @ 0x00000000071d28ec ... 2024.07.18 03:58:29.000900 [ 48468 ] {8cf63d7e-dcbf-4af6-bd7c-0e1789ddce3b} executeQuery: (from [::1]:40410) (comment: 01747_executable_pool_dictionary_implicit_key.sql) SELECT dictGet('executable_pool_simple_implicit_key', 'a', toUInt64(1)); (stage: Complete) # and no more rows for 8cf63d7e-dcbf-4af6-bd7c-0e1789ddce3b [1]: https://s3.amazonaws.com/clickhouse-test-reports/66495/bc029ed8207ac75e96e9cb48cb79d27a9ffa4e2f/stress_test__debug_.html The problem that it should be properly cancelled, otherwise it will not be loaded in loadImpl(), but will be waited. Signed-off-by: Azat Khuzhin --- src/Interpreters/ExternalLoader.cpp | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ExternalLoader.cpp b/src/Interpreters/ExternalLoader.cpp index 96405f35f3f..511300be2e0 100644 --- a/src/Interpreters/ExternalLoader.cpp +++ b/src/Interpreters/ExternalLoader.cpp @@ -922,7 +922,16 @@ private: if (enable_async_loading) { /// Put a job to the thread pool for the loading. - auto thread = ThreadFromGlobalPool{&LoadingDispatcher::doLoading, this, info.name, loading_id, forced_to_reload, min_id_to_finish_loading_dependencies_, true, CurrentThread::getGroup()}; + ThreadFromGlobalPool thread; + try + { + thread = ThreadFromGlobalPool{&LoadingDispatcher::doLoading, this, info.name, loading_id, forced_to_reload, min_id_to_finish_loading_dependencies_, true, CurrentThread::getGroup()}; + } + catch (...) + { + cancelLoading(info); + throw; + } loading_threads.try_emplace(loading_id, std::move(thread)); } else From a433115434ebe4c1d69f2ed200005fa93c7adcb7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 19:18:00 +0200 Subject: [PATCH 25/27] Fix typo --- docker/test/util/process_functional_tests_result.py | 6 +++--- docs/changelogs/v20.5.1.3833-prestable.md | 2 +- docs/changelogs/v21.11.1.8636-prestable.md | 6 +++--- docs/changelogs/v21.12.1.9017-prestable.md | 10 +++++----- docs/changelogs/v21.5.1.6601-prestable.md | 2 +- docs/changelogs/v21.6.9.7-stable.md | 2 +- docs/changelogs/v21.7.9.7-stable.md | 2 +- docs/changelogs/v21.8.1.7409-prestable.md | 2 +- docs/changelogs/v21.8.5.7-lts.md | 2 +- docs/changelogs/v21.9.1.8000-prestable.md | 2 +- docs/changelogs/v22.1.1.2542-prestable.md | 4 ++-- docs/changelogs/v22.4.1.2305-prestable.md | 2 +- docs/changelogs/v23.4.1.1943-stable.md | 3 +-- docs/changelogs/v23.6.1.1524-stable.md | 2 +- docs/zh/changelog/index.md | 2 +- src/Common/SystemLogBase.cpp | 2 +- 16 files changed, 25 insertions(+), 26 deletions(-) diff --git a/docker/test/util/process_functional_tests_result.py b/docker/test/util/process_functional_tests_result.py index 3da1a8f3674..aa2ea686c46 100755 --- a/docker/test/util/process_functional_tests_result.py +++ b/docker/test/util/process_functional_tests_result.py @@ -161,11 +161,11 @@ def process_result(result_path, broken_tests): retries, test_results, ) = process_test_log(result_path, broken_tests) - is_flacky_check = 1 < int(os.environ.get("NUM_TRIES", 1)) - logging.info("Is flaky check: %s", is_flacky_check) + is_flaky_check = 1 < int(os.environ.get("NUM_TRIES", 1)) + logging.info("Is flaky check: %s", is_flaky_check) # If no tests were run (success == 0) it indicates an error (e.g. server did not start or crashed immediately) # But it's Ok for "flaky checks" - they can contain just one test for check which is marked as skipped. - if failed != 0 or unknown != 0 or (success == 0 and (not is_flacky_check)): + if failed != 0 or unknown != 0 or (success == 0 and (not is_flaky_check)): state = "failure" if hung: diff --git a/docs/changelogs/v20.5.1.3833-prestable.md b/docs/changelogs/v20.5.1.3833-prestable.md index 79d61fb12f9..3de263f5e37 100644 --- a/docs/changelogs/v20.5.1.3833-prestable.md +++ b/docs/changelogs/v20.5.1.3833-prestable.md @@ -331,7 +331,7 @@ * Fix several non significant errors in unit tests. [#11262](https://github.com/ClickHouse/ClickHouse/pull/11262) ([alesapin](https://github.com/alesapin)). * Add a test for Join table engine from @donmikel. This closes [#9158](https://github.com/ClickHouse/ClickHouse/issues/9158). [#11265](https://github.com/ClickHouse/ClickHouse/pull/11265) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Repeat test in CI if `curl` invocation was timed out. It is possible due to system hangups for 10+ seconds that are typical in our CI infrastructure. This fixes [#11267](https://github.com/ClickHouse/ClickHouse/issues/11267). [#11268](https://github.com/ClickHouse/ClickHouse/pull/11268) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fix potentially flacky test `00731_long_merge_tree_select_opened_files.sh`. It does not fail frequently but we have discovered potential race condition in this test while experimenting with ThreadFuzzer: [#9814](https://github.com/ClickHouse/ClickHouse/issues/9814) See [link](https://clickhouse-test-reports.s3.yandex.net/9814/40e3023e215df22985d275bf85f4d2290897b76b/functional_stateless_tests_(unbundled).html#fail1) for the example. [#11270](https://github.com/ClickHouse/ClickHouse/pull/11270) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fix potentially flaky test `00731_long_merge_tree_select_opened_files.sh`. It does not fail frequently but we have discovered potential race condition in this test while experimenting with ThreadFuzzer: [#9814](https://github.com/ClickHouse/ClickHouse/issues/9814) See [link](https://clickhouse-test-reports.s3.yandex.net/9814/40e3023e215df22985d275bf85f4d2290897b76b/functional_stateless_tests_(unbundled).html#fail1) for the example. [#11270](https://github.com/ClickHouse/ClickHouse/pull/11270) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Now clickhouse-test check the server aliveness before tests run. [#11285](https://github.com/ClickHouse/ClickHouse/pull/11285) ([alesapin](https://github.com/alesapin)). * Emit a warning if server was build in debug or with sanitizers. [#11304](https://github.com/ClickHouse/ClickHouse/pull/11304) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Better check for hung queries in clickhouse-test. [#11321](https://github.com/ClickHouse/ClickHouse/pull/11321) ([Alexey Milovidov](https://github.com/alexey-milovidov)). diff --git a/docs/changelogs/v21.11.1.8636-prestable.md b/docs/changelogs/v21.11.1.8636-prestable.md index d6a435dd3ce..95c8580d591 100644 --- a/docs/changelogs/v21.11.1.8636-prestable.md +++ b/docs/changelogs/v21.11.1.8636-prestable.md @@ -280,7 +280,7 @@ sidebar_label: 2022 * Cleanup unbundled image [#29689](https://github.com/ClickHouse/ClickHouse/pull/29689) ([Azat Khuzhin](https://github.com/azat)). * Fix memory tracking for merges and mutations [#29691](https://github.com/ClickHouse/ClickHouse/pull/29691) ([Azat Khuzhin](https://github.com/azat)). * Fix data-race in WriteIndirectBuffer (used in DiskMemory) [#29692](https://github.com/ClickHouse/ClickHouse/pull/29692) ([Azat Khuzhin](https://github.com/azat)). -* Fix flacky test [#29706](https://github.com/ClickHouse/ClickHouse/pull/29706) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#29706](https://github.com/ClickHouse/ClickHouse/pull/29706) ([Kseniia Sumarokova](https://github.com/kssenii)). * BorrowedObjectPool condition variable notify fix [#29722](https://github.com/ClickHouse/ClickHouse/pull/29722) ([Maksim Kita](https://github.com/kitaisreal)). * Better exception message for local interactive [#29737](https://github.com/ClickHouse/ClickHouse/pull/29737) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix --stage for clickhouse-local [#29745](https://github.com/ClickHouse/ClickHouse/pull/29745) ([Azat Khuzhin](https://github.com/azat)). @@ -308,7 +308,7 @@ sidebar_label: 2022 * Fix client [#29864](https://github.com/ClickHouse/ClickHouse/pull/29864) ([Kseniia Sumarokova](https://github.com/kssenii)). * Remove some more streams. [#29898](https://github.com/ClickHouse/ClickHouse/pull/29898) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Add logging in ZooKeeper client [#29901](https://github.com/ClickHouse/ClickHouse/pull/29901) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Fix some flacky tests [#29902](https://github.com/ClickHouse/ClickHouse/pull/29902) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix some flaky tests [#29902](https://github.com/ClickHouse/ClickHouse/pull/29902) ([Kseniia Sumarokova](https://github.com/kssenii)). * Grep server log even if it contains binary data [#29903](https://github.com/ClickHouse/ClickHouse/pull/29903) ([Alexander Tokmakov](https://github.com/tavplubix)). * Cosmetic refactoring of server constants. [#29913](https://github.com/ClickHouse/ClickHouse/pull/29913) ([Amos Bird](https://github.com/amosbird)). * Format improvement of AlterQuery [#29916](https://github.com/ClickHouse/ClickHouse/pull/29916) ([flynn](https://github.com/ucasfl)). @@ -465,7 +465,7 @@ sidebar_label: 2022 * Fix docs release [#30933](https://github.com/ClickHouse/ClickHouse/pull/30933) ([alesapin](https://github.com/alesapin)). * Fix style check [#30937](https://github.com/ClickHouse/ClickHouse/pull/30937) ([alesapin](https://github.com/alesapin)). * Fix file progress for clickhouse-local [#30938](https://github.com/ClickHouse/ClickHouse/pull/30938) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix flacky test [#30940](https://github.com/ClickHouse/ClickHouse/pull/30940) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#30940](https://github.com/ClickHouse/ClickHouse/pull/30940) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix reading from TinyLog [#30941](https://github.com/ClickHouse/ClickHouse/pull/30941) ([Vitaly Baranov](https://github.com/vitlibar)). * Add github to known hosts in docs release [#30947](https://github.com/ClickHouse/ClickHouse/pull/30947) ([alesapin](https://github.com/alesapin)). * Parse json from response in ci checks [#30948](https://github.com/ClickHouse/ClickHouse/pull/30948) ([alesapin](https://github.com/alesapin)). diff --git a/docs/changelogs/v21.12.1.9017-prestable.md b/docs/changelogs/v21.12.1.9017-prestable.md index bd84873e67a..f5416664d35 100644 --- a/docs/changelogs/v21.12.1.9017-prestable.md +++ b/docs/changelogs/v21.12.1.9017-prestable.md @@ -220,7 +220,7 @@ sidebar_label: 2022 * Fix test_backward_compatibility [#30950](https://github.com/ClickHouse/ClickHouse/pull/30950) ([Ilya Yatsishin](https://github.com/qoega)). * Add stress test to github actions [#30952](https://github.com/ClickHouse/ClickHouse/pull/30952) ([alesapin](https://github.com/alesapin)). * Try smaller blacklist of non parallel integration tests [#30963](https://github.com/ClickHouse/ClickHouse/pull/30963) ([Ilya Yatsishin](https://github.com/qoega)). -* Fix flacky test [#30967](https://github.com/ClickHouse/ClickHouse/pull/30967) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#30967](https://github.com/ClickHouse/ClickHouse/pull/30967) ([Kseniia Sumarokova](https://github.com/kssenii)). * Move access-rights source code [#30973](https://github.com/ClickHouse/ClickHouse/pull/30973) ([Vitaly Baranov](https://github.com/vitlibar)). * Set output_format_avro_rows_in_file default to 1 [#30990](https://github.com/ClickHouse/ClickHouse/pull/30990) ([Kruglov Pavel](https://github.com/Avogar)). * Remove remaining usages of Y_IGNORE [#30993](https://github.com/ClickHouse/ClickHouse/pull/30993) ([Yuriy Chernyshov](https://github.com/georgthegreat)). @@ -353,7 +353,7 @@ sidebar_label: 2022 * Support toUInt8/toInt8 for if constant condition optimization. [#31866](https://github.com/ClickHouse/ClickHouse/pull/31866) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Added -no-sanitize=unsigned-integer-overflow build flag [#31881](https://github.com/ClickHouse/ClickHouse/pull/31881) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). * Fix typos [#31886](https://github.com/ClickHouse/ClickHouse/pull/31886) ([Anton Popov](https://github.com/CurtizJ)). -* Try to fix flacky test. [#31889](https://github.com/ClickHouse/ClickHouse/pull/31889) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Try to fix flaky test. [#31889](https://github.com/ClickHouse/ClickHouse/pull/31889) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Reduce the files that depend on parser headers [#31896](https://github.com/ClickHouse/ClickHouse/pull/31896) ([Raúl Marín](https://github.com/Algunenano)). * Fix magic_enum for debug helpers (fixes build w/ USE_DEBUG_HELPERS) [#31922](https://github.com/ClickHouse/ClickHouse/pull/31922) ([Azat Khuzhin](https://github.com/azat)). * Remove some trash from build [#31923](https://github.com/ClickHouse/ClickHouse/pull/31923) ([Alexey Milovidov](https://github.com/alexey-milovidov)). @@ -387,7 +387,7 @@ sidebar_label: 2022 * make looping in H3 funcs uniform [#32110](https://github.com/ClickHouse/ClickHouse/pull/32110) ([Bharat Nallan](https://github.com/bharatnc)). * Remove PVS check from master [#32114](https://github.com/ClickHouse/ClickHouse/pull/32114) ([alesapin](https://github.com/alesapin)). * Fix flaky keeper whitelist test [#32115](https://github.com/ClickHouse/ClickHouse/pull/32115) ([alesapin](https://github.com/alesapin)). -* Fix flacky test test_executable_storage_input [#32118](https://github.com/ClickHouse/ClickHouse/pull/32118) ([Maksim Kita](https://github.com/kitaisreal)). +* Fix flaky test test_executable_storage_input [#32118](https://github.com/ClickHouse/ClickHouse/pull/32118) ([Maksim Kita](https://github.com/kitaisreal)). * Fix data race in `removePartAndEnqueueFetch(...)` [#32119](https://github.com/ClickHouse/ClickHouse/pull/32119) ([Alexander Tokmakov](https://github.com/tavplubix)). * Move fuzzers and unit tests to another group [#32120](https://github.com/ClickHouse/ClickHouse/pull/32120) ([alesapin](https://github.com/alesapin)). * Add a test with 20000 mutations in one query [#32122](https://github.com/ClickHouse/ClickHouse/pull/32122) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). @@ -411,11 +411,11 @@ sidebar_label: 2022 * Add test for [#32186](https://github.com/ClickHouse/ClickHouse/issues/32186) [#32203](https://github.com/ClickHouse/ClickHouse/pull/32203) ([Raúl Marín](https://github.com/Algunenano)). * Fix uncaught exception in DatabaseLazy [#32206](https://github.com/ClickHouse/ClickHouse/pull/32206) ([Alexander Tokmakov](https://github.com/tavplubix)). * Update ASTCreateQuery.cpp [#32208](https://github.com/ClickHouse/ClickHouse/pull/32208) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix flacky fileLog test (probably) [#32209](https://github.com/ClickHouse/ClickHouse/pull/32209) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky fileLog test (probably) [#32209](https://github.com/ClickHouse/ClickHouse/pull/32209) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix jemalloc under osx [#32219](https://github.com/ClickHouse/ClickHouse/pull/32219) ([Azat Khuzhin](https://github.com/azat)). * Add missing timezones to some tests [#32222](https://github.com/ClickHouse/ClickHouse/pull/32222) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix versioning of aggregate functions (fixes performance tests) [#32236](https://github.com/ClickHouse/ClickHouse/pull/32236) ([Azat Khuzhin](https://github.com/azat)). -* Disable window view tests temporarily because still flacky [#32257](https://github.com/ClickHouse/ClickHouse/pull/32257) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Disable window view tests temporarily because still flaky [#32257](https://github.com/ClickHouse/ClickHouse/pull/32257) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix typo in tupleToNameValuePairs doc [#32262](https://github.com/ClickHouse/ClickHouse/pull/32262) ([Vladimir C](https://github.com/vdimir)). * Fix possible Pipeline stuck in case of StrictResize processor. [#32270](https://github.com/ClickHouse/ClickHouse/pull/32270) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix possible crash in DataTypeAggregateFunction [#32287](https://github.com/ClickHouse/ClickHouse/pull/32287) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). diff --git a/docs/changelogs/v21.5.1.6601-prestable.md b/docs/changelogs/v21.5.1.6601-prestable.md index b7dd8ae87c0..69ea9cb8d0e 100644 --- a/docs/changelogs/v21.5.1.6601-prestable.md +++ b/docs/changelogs/v21.5.1.6601-prestable.md @@ -158,7 +158,7 @@ sidebar_label: 2022 * MemoryStorage sync comments and code [#22721](https://github.com/ClickHouse/ClickHouse/pull/22721) ([Maksim Kita](https://github.com/kitaisreal)). * Fix potential segfault on Keeper startup [#22743](https://github.com/ClickHouse/ClickHouse/pull/22743) ([alesapin](https://github.com/alesapin)). * Avoid using harmful function rand() [#22744](https://github.com/ClickHouse/ClickHouse/pull/22744) ([Amos Bird](https://github.com/amosbird)). -* Fix flacky hedged tests [#22746](https://github.com/ClickHouse/ClickHouse/pull/22746) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix flaky hedged tests [#22746](https://github.com/ClickHouse/ClickHouse/pull/22746) ([Kruglov Pavel](https://github.com/Avogar)). * add more messages when flushing the logs [#22761](https://github.com/ClickHouse/ClickHouse/pull/22761) ([Alexander Kuzmenkov](https://github.com/akuzm)). * Moved BorrowedObjectPool to common [#22764](https://github.com/ClickHouse/ClickHouse/pull/22764) ([Maksim Kita](https://github.com/kitaisreal)). * Functions ExternalDictionaries standardize exception throw [#22821](https://github.com/ClickHouse/ClickHouse/pull/22821) ([Maksim Kita](https://github.com/kitaisreal)). diff --git a/docs/changelogs/v21.6.9.7-stable.md b/docs/changelogs/v21.6.9.7-stable.md index 0a989e4d6b7..533c58badac 100644 --- a/docs/changelogs/v21.6.9.7-stable.md +++ b/docs/changelogs/v21.6.9.7-stable.md @@ -55,7 +55,7 @@ sidebar_label: 2022 * Try fix rabbitmq tests [#26826](https://github.com/ClickHouse/ClickHouse/pull/26826) ([Kseniia Sumarokova](https://github.com/kssenii)). * One more library bridge fix [#26873](https://github.com/ClickHouse/ClickHouse/pull/26873) ([Kseniia Sumarokova](https://github.com/kssenii)). * Update PVS checksum [#27317](https://github.com/ClickHouse/ClickHouse/pull/27317) ([Alexander Tokmakov](https://github.com/tavplubix)). -* Fix flacky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix throw without exception in MySQL source. [#28027](https://github.com/ClickHouse/ClickHouse/pull/28027) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix race between REPLACE PARTITION and MOVE PARTITION [#28035](https://github.com/ClickHouse/ClickHouse/pull/28035) ([Alexander Tokmakov](https://github.com/tavplubix)). * Follow-up to [#28016](https://github.com/ClickHouse/ClickHouse/issues/28016) [#28036](https://github.com/ClickHouse/ClickHouse/pull/28036) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v21.7.9.7-stable.md b/docs/changelogs/v21.7.9.7-stable.md index 7aaab54af6b..684d0e8995e 100644 --- a/docs/changelogs/v21.7.9.7-stable.md +++ b/docs/changelogs/v21.7.9.7-stable.md @@ -35,7 +35,7 @@ sidebar_label: 2022 #### NOT FOR CHANGELOG / INSIGNIFICANT * Fix prometheus metric name [#26140](https://github.com/ClickHouse/ClickHouse/pull/26140) ([Vladimir C](https://github.com/vdimir)). -* Fix flacky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix throw without exception in MySQL source. [#28027](https://github.com/ClickHouse/ClickHouse/pull/28027) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix race between REPLACE PARTITION and MOVE PARTITION [#28035](https://github.com/ClickHouse/ClickHouse/pull/28035) ([Alexander Tokmakov](https://github.com/tavplubix)). * Follow-up to [#28016](https://github.com/ClickHouse/ClickHouse/issues/28016) [#28036](https://github.com/ClickHouse/ClickHouse/pull/28036) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v21.8.1.7409-prestable.md b/docs/changelogs/v21.8.1.7409-prestable.md index cb6ab82b30f..6ef2f1b50d2 100644 --- a/docs/changelogs/v21.8.1.7409-prestable.md +++ b/docs/changelogs/v21.8.1.7409-prestable.md @@ -101,7 +101,7 @@ sidebar_label: 2022 * Separate log files for separate runs in stress test [#25741](https://github.com/ClickHouse/ClickHouse/pull/25741) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Fix slow performance test [#25742](https://github.com/ClickHouse/ClickHouse/pull/25742) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * DatabaseAtomic EXCHANGE DICTIONARIES fix test [#25753](https://github.com/ClickHouse/ClickHouse/pull/25753) ([Maksim Kita](https://github.com/kitaisreal)). -* Try fix flacky rabbitmq test [#25756](https://github.com/ClickHouse/ClickHouse/pull/25756) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Try fix flaky rabbitmq test [#25756](https://github.com/ClickHouse/ClickHouse/pull/25756) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add a test for [#13993](https://github.com/ClickHouse/ClickHouse/issues/13993) [#25758](https://github.com/ClickHouse/ClickHouse/pull/25758) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Set follow-fork-mode child for gdb in stress/fasttest/fuzzer [#25769](https://github.com/ClickHouse/ClickHouse/pull/25769) ([Azat Khuzhin](https://github.com/azat)). * Ignore TOO_DEEP_RECURSION server exception during fuzzing [#25770](https://github.com/ClickHouse/ClickHouse/pull/25770) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v21.8.5.7-lts.md b/docs/changelogs/v21.8.5.7-lts.md index fa459e093f7..4d0727e362c 100644 --- a/docs/changelogs/v21.8.5.7-lts.md +++ b/docs/changelogs/v21.8.5.7-lts.md @@ -40,7 +40,7 @@ sidebar_label: 2022 * Fix several bugs in ZooKeeper snapshots deserialization [#26127](https://github.com/ClickHouse/ClickHouse/pull/26127) ([alesapin](https://github.com/alesapin)). * Fix prometheus metric name [#26140](https://github.com/ClickHouse/ClickHouse/pull/26140) ([Vladimir C](https://github.com/vdimir)). -* Fix flacky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix throw without exception in MySQL source. [#28027](https://github.com/ClickHouse/ClickHouse/pull/28027) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix race between REPLACE PARTITION and MOVE PARTITION [#28035](https://github.com/ClickHouse/ClickHouse/pull/28035) ([Alexander Tokmakov](https://github.com/tavplubix)). * Follow-up to [#28016](https://github.com/ClickHouse/ClickHouse/issues/28016) [#28036](https://github.com/ClickHouse/ClickHouse/pull/28036) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v21.9.1.8000-prestable.md b/docs/changelogs/v21.9.1.8000-prestable.md index bc921a68693..70ae3697e92 100644 --- a/docs/changelogs/v21.9.1.8000-prestable.md +++ b/docs/changelogs/v21.9.1.8000-prestable.md @@ -346,7 +346,7 @@ sidebar_label: 2022 * Update PVS checksum [#27317](https://github.com/ClickHouse/ClickHouse/pull/27317) ([Alexander Tokmakov](https://github.com/tavplubix)). * Fix 01300_client_save_history_when_terminated_long [#27324](https://github.com/ClickHouse/ClickHouse/pull/27324) ([Raúl Marín](https://github.com/Algunenano)). * Try update contrib/zlib-ng [#27327](https://github.com/ClickHouse/ClickHouse/pull/27327) ([Ilya Yatsishin](https://github.com/qoega)). -* Fix flacky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test [#27383](https://github.com/ClickHouse/ClickHouse/pull/27383) ([Kseniia Sumarokova](https://github.com/kssenii)). * Add and check system.mutations for database filter [#27384](https://github.com/ClickHouse/ClickHouse/pull/27384) ([Azat Khuzhin](https://github.com/azat)). * Correct the key data type used in mapContains [#27423](https://github.com/ClickHouse/ClickHouse/pull/27423) ([Fuwang Hu](https://github.com/fuwhu)). * Fix tests for WithMergeableStateAfterAggregationAndLimit [#27424](https://github.com/ClickHouse/ClickHouse/pull/27424) ([Azat Khuzhin](https://github.com/azat)). diff --git a/docs/changelogs/v22.1.1.2542-prestable.md b/docs/changelogs/v22.1.1.2542-prestable.md index cacd13c1e12..3b0422abb11 100644 --- a/docs/changelogs/v22.1.1.2542-prestable.md +++ b/docs/changelogs/v22.1.1.2542-prestable.md @@ -398,7 +398,7 @@ sidebar_label: 2022 * test for [#24410](https://github.com/ClickHouse/ClickHouse/issues/24410) [#33265](https://github.com/ClickHouse/ClickHouse/pull/33265) ([Denny Crane](https://github.com/den-crane)). * Wait for RabbitMQ container to actually start when it was restarted in test on purpose [#33266](https://github.com/ClickHouse/ClickHouse/pull/33266) ([Kseniia Sumarokova](https://github.com/kssenii)). * Mark max_alter_threads as obsolete [#33268](https://github.com/ClickHouse/ClickHouse/pull/33268) ([Denny Crane](https://github.com/den-crane)). -* Fix azure tests flackyness because of azure server closing connection [#33269](https://github.com/ClickHouse/ClickHouse/pull/33269) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix azure tests flakyness because of azure server closing connection [#33269](https://github.com/ClickHouse/ClickHouse/pull/33269) ([Kseniia Sumarokova](https://github.com/kssenii)). * Test for [#26920](https://github.com/ClickHouse/ClickHouse/issues/26920) [#33272](https://github.com/ClickHouse/ClickHouse/pull/33272) ([Denny Crane](https://github.com/den-crane)). * Fix test_storage_kafka failures by adjusting retention.ms [#33278](https://github.com/ClickHouse/ClickHouse/pull/33278) ([Azat Khuzhin](https://github.com/azat)). * Disable FunctionConvertFromString::canBeExecutedOnDefaultArguments [#33286](https://github.com/ClickHouse/ClickHouse/pull/33286) ([Vladimir C](https://github.com/vdimir)). @@ -447,7 +447,7 @@ sidebar_label: 2022 * Update mongodb.md [#33585](https://github.com/ClickHouse/ClickHouse/pull/33585) ([Kseniia Sumarokova](https://github.com/kssenii)). * Restore existing static builds links [#33597](https://github.com/ClickHouse/ClickHouse/pull/33597) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix pylint for run_check.py [#33600](https://github.com/ClickHouse/ClickHouse/pull/33600) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Fix flacky test_dictionaries_postgresql/ [#33601](https://github.com/ClickHouse/ClickHouse/pull/33601) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test_dictionaries_postgresql/ [#33601](https://github.com/ClickHouse/ClickHouse/pull/33601) ([Kseniia Sumarokova](https://github.com/kssenii)). * Make ZooKeeper client better interpret keeper server connection reject [#33602](https://github.com/ClickHouse/ClickHouse/pull/33602) ([alesapin](https://github.com/alesapin)). * Fix broken workflow dependencies [#33608](https://github.com/ClickHouse/ClickHouse/pull/33608) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Force rebuild images in CI [#33609](https://github.com/ClickHouse/ClickHouse/pull/33609) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). diff --git a/docs/changelogs/v22.4.1.2305-prestable.md b/docs/changelogs/v22.4.1.2305-prestable.md index b277137ca7e..e8304e6505f 100644 --- a/docs/changelogs/v22.4.1.2305-prestable.md +++ b/docs/changelogs/v22.4.1.2305-prestable.md @@ -410,7 +410,7 @@ sidebar_label: 2022 * Fix mongodb test with new cert [#36161](https://github.com/ClickHouse/ClickHouse/pull/36161) ([alesapin](https://github.com/alesapin)). * Some fixes for ReplicatedMergeTree [#36163](https://github.com/ClickHouse/ClickHouse/pull/36163) ([Alexander Tokmakov](https://github.com/tavplubix)). * clickhouse-client: properly cancel query in case of error during formatting data [#36164](https://github.com/ClickHouse/ClickHouse/pull/36164) ([Azat Khuzhin](https://github.com/azat)). -* Fix flacky test 01161_all_system_tables under s3 storage [#36175](https://github.com/ClickHouse/ClickHouse/pull/36175) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix flaky test 01161_all_system_tables under s3 storage [#36175](https://github.com/ClickHouse/ClickHouse/pull/36175) ([Kseniia Sumarokova](https://github.com/kssenii)). * Revert "Fix possible mutation stuck due to race with DROP_RANGE" [#36190](https://github.com/ClickHouse/ClickHouse/pull/36190) ([Azat Khuzhin](https://github.com/azat)). * Use atomic instead of mutex + condvar in ParallelReadBuffer [#36192](https://github.com/ClickHouse/ClickHouse/pull/36192) ([Kruglov Pavel](https://github.com/Avogar)). * Follow-up to [#36138](https://github.com/ClickHouse/ClickHouse/issues/36138) [#36194](https://github.com/ClickHouse/ClickHouse/pull/36194) ([Alexander Tokmakov](https://github.com/tavplubix)). diff --git a/docs/changelogs/v23.4.1.1943-stable.md b/docs/changelogs/v23.4.1.1943-stable.md index ea16f5856be..34590ba9d37 100644 --- a/docs/changelogs/v23.4.1.1943-stable.md +++ b/docs/changelogs/v23.4.1.1943-stable.md @@ -321,7 +321,7 @@ sidebar_label: 2023 * Add a test for [#38128](https://github.com/ClickHouse/ClickHouse/issues/38128) [#48817](https://github.com/ClickHouse/ClickHouse/pull/48817) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Remove excessive logging [#48826](https://github.com/ClickHouse/ClickHouse/pull/48826) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * remove duplicate indentwith in clang-format [#48834](https://github.com/ClickHouse/ClickHouse/pull/48834) ([cluster](https://github.com/infdahai)). -* Try fix flacky test_concurrent_alter_move_and_drop [#48843](https://github.com/ClickHouse/ClickHouse/pull/48843) ([Sergei Trifonov](https://github.com/serxa)). +* Try fix flaky test_concurrent_alter_move_and_drop [#48843](https://github.com/ClickHouse/ClickHouse/pull/48843) ([Sergei Trifonov](https://github.com/serxa)). * fix the race wait loading parts [#48844](https://github.com/ClickHouse/ClickHouse/pull/48844) ([Sema Checherinda](https://github.com/CheSema)). * suppress assert of progress for test_system_replicated_fetches [#48856](https://github.com/ClickHouse/ClickHouse/pull/48856) ([Han Fei](https://github.com/hanfei1991)). * Fix: do not run test_store_cleanup_disk_s3 in parallel [#48863](https://github.com/ClickHouse/ClickHouse/pull/48863) ([Igor Nikonov](https://github.com/devcrafter)). @@ -372,4 +372,3 @@ sidebar_label: 2023 * suppress two timeout tests [#49175](https://github.com/ClickHouse/ClickHouse/pull/49175) ([Han Fei](https://github.com/hanfei1991)). * Document makeDateTime() and its variants [#49183](https://github.com/ClickHouse/ClickHouse/pull/49183) ([Robert Schulze](https://github.com/rschu1ze)). * Fix after [#49110](https://github.com/ClickHouse/ClickHouse/issues/49110) [#49206](https://github.com/ClickHouse/ClickHouse/pull/49206) ([Kseniia Sumarokova](https://github.com/kssenii)). - diff --git a/docs/changelogs/v23.6.1.1524-stable.md b/docs/changelogs/v23.6.1.1524-stable.md index b91c5340789..0de9ab37653 100644 --- a/docs/changelogs/v23.6.1.1524-stable.md +++ b/docs/changelogs/v23.6.1.1524-stable.md @@ -263,7 +263,7 @@ sidebar_label: 2023 * Fix broken labeling for `manual approve` [#51405](https://github.com/ClickHouse/ClickHouse/pull/51405) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). * Fix parts lifetime in `MergeTreeTransaction` [#51407](https://github.com/ClickHouse/ClickHouse/pull/51407) ([Alexander Tokmakov](https://github.com/tavplubix)). * Fix flaky test test_skip_empty_files [#51409](https://github.com/ClickHouse/ClickHouse/pull/51409) ([Kruglov Pavel](https://github.com/Avogar)). -* fix flacky test test_profile_events_s3 [#51412](https://github.com/ClickHouse/ClickHouse/pull/51412) ([Sema Checherinda](https://github.com/CheSema)). +* fix flaky test test_profile_events_s3 [#51412](https://github.com/ClickHouse/ClickHouse/pull/51412) ([Sema Checherinda](https://github.com/CheSema)). * Update README.md [#51413](https://github.com/ClickHouse/ClickHouse/pull/51413) ([Tyler Hannan](https://github.com/tylerhannan)). * Replace try/catch logic in hasTokenOrNull() by something more lightweight [#51425](https://github.com/ClickHouse/ClickHouse/pull/51425) ([Robert Schulze](https://github.com/rschu1ze)). * Add retries to `tlsv1_3` tests [#51434](https://github.com/ClickHouse/ClickHouse/pull/51434) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)). diff --git a/docs/zh/changelog/index.md b/docs/zh/changelog/index.md index cd77a8c03cf..fb50dfcee85 100644 --- a/docs/zh/changelog/index.md +++ b/docs/zh/changelog/index.md @@ -252,7 +252,7 @@ sidebar_label: "\u53D8\u66F4\u65E5\u5FD7" - 抑制MSan下的一些测试失败。 [#8780](https://github.com/ClickHouse/ClickHouse/pull/8780) ([Alexander Kuzmenkov](https://github.com/akuzm)) - 加速 “exception while insert” 测试 此测试通常在具有复盖率的调试版本中超时。 [#8711](https://github.com/ClickHouse/ClickHouse/pull/8711) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) - 更新 `libcxx` 和 `libcxxabi` 为了主人 在准备 [#9304](https://github.com/ClickHouse/ClickHouse/issues/9304) [#9308](https://github.com/ClickHouse/ClickHouse/pull/9308) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) -- 修复flacky测试 `00910_zookeeper_test_alter_compression_codecs`. [#9525](https://github.com/ClickHouse/ClickHouse/pull/9525) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) +- 修复flaky测试 `00910_zookeeper_test_alter_compression_codecs`. [#9525](https://github.com/ClickHouse/ClickHouse/pull/9525) ([阿列克谢-米洛维多夫](https://github.com/alexey-milovidov)) - 清理重复的链接器标志。 确保链接器不会查找意想不到的符号。 [#9433](https://github.com/ClickHouse/ClickHouse/pull/9433) ([阿莫斯鸟](https://github.com/amosbird)) - 添加 `clickhouse-odbc` 驱动程序进入测试图像。 这允许通过自己的ODBC驱动程序测试ClickHouse与ClickHouse的交互。 [#9348](https://github.com/ClickHouse/ClickHouse/pull/9348) ([filimonov](https://github.com/filimonov)) - 修复单元测试中的几个错误。 [#9047](https://github.com/ClickHouse/ClickHouse/pull/9047) ([阿利沙平](https://github.com/alesapin)) diff --git a/src/Common/SystemLogBase.cpp b/src/Common/SystemLogBase.cpp index a9307c3be99..7d2c15714e2 100644 --- a/src/Common/SystemLogBase.cpp +++ b/src/Common/SystemLogBase.cpp @@ -65,7 +65,7 @@ void SystemLogQueue::push(LogElement&& element) /// Memory can be allocated while resizing on queue.push_back. /// The size of allocation can be in order of a few megabytes. /// But this should not be accounted for query memory usage. - /// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flacky. + /// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flaky. MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; /// Should not log messages under mutex. From a573b2926e13ea31c7947b0429b1b5723c7fb938 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 4 Aug 2024 23:59:01 +0200 Subject: [PATCH 26/27] Fixes for the script --- docker/test/stateless/run.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index da17b82d91b..8b9e729970c 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -72,8 +72,12 @@ if [[ -n "$BUGFIX_VALIDATE_CHECK" ]] && [[ "$BUGFIX_VALIDATE_CHECK" -eq 1 ]]; th remove_keeper_config "latest_logs_cache_size_threshold" "[[:digit:]]\+" fi +export IS_FLAKY_CHECK=0 + # For flaky check we also enable thread fuzzer if [ "$NUM_TRIES" -gt "1" ]; then + export IS_FLAKY_CHECK=1 + export THREAD_FUZZER_CPU_TIME_PERIOD_US=1000 export THREAD_FUZZER_SLEEP_PROBABILITY=0.1 export THREAD_FUZZER_SLEEP_TIME_US_MAX=100000 From 538761b43dbf704d0700548e61b4034ef66c5766 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 5 Aug 2024 00:02:32 +0200 Subject: [PATCH 27/27] Fix flaky check --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index b70dd61a25a..38b0e99760e 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -1232,7 +1232,7 @@ class TestCase: ): return FailureReason.SKIP - elif "no-flaky-check" in tags and (args.test_runs > 1): + elif "no-flaky-check" in tags and (1 == int(os.environ.get("IS_FLAKY_CHECK", 0))): return FailureReason.SKIP elif tags: